From b353d8891dbe001e1828923fb4379f046a29843e Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Fri, 17 Oct 2014 17:05:29 +0300 Subject: [PATCH 01/12] working --- config/client.ssl.properties | 9 + config/server.properties | 4 +- config/server.ssl.properties | 32 + .../main/scala/kafka/network/ChannelFactory.scala | 43 ++ .../main/scala/kafka/network/KafkaChannel.scala | 54 ++ .../kafka/network/PlainSocketChannelFactory.scala | 27 + .../main/scala/kafka/network/SocketServer.scala | 130 ++-- .../main/scala/kafka/network/ssl/KeyStores.scala | 35 ++ .../src/main/scala/kafka/network/ssl/SSLAuth.scala | 53 ++ .../kafka/network/ssl/SSLChannelFactory.scala | 31 + .../kafka/network/ssl/SSLConnectionConfig.scala | 57 ++ .../scala/kafka/network/ssl/SSLSocketChannel.scala | 661 +++++++++++++++++++++ .../kafka/network/ssl/store/JKSInitializer.scala | 62 ++ core/src/main/scala/kafka/server/KafkaConfig.scala | 12 +- core/src/main/scala/kafka/server/KafkaServer.scala | 45 +- 15 files changed, 1198 insertions(+), 57 deletions(-) create mode 100644 config/client.ssl.properties create mode 100644 config/server.ssl.properties create mode 100644 core/src/main/scala/kafka/network/ChannelFactory.scala create mode 100644 core/src/main/scala/kafka/network/KafkaChannel.scala create mode 100644 core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala create mode 100644 core/src/main/scala/kafka/network/ssl/KeyStores.scala create mode 100644 core/src/main/scala/kafka/network/ssl/SSLAuth.scala create mode 100644 core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala create mode 100644 core/src/main/scala/kafka/network/ssl/SSLConnectionConfig.scala create mode 100644 core/src/main/scala/kafka/network/ssl/SSLSocketChannel.scala create mode 100644 core/src/main/scala/kafka/network/ssl/store/JKSInitializer.scala diff --git a/config/client.ssl.properties b/config/client.ssl.properties new file mode 100644 index 0000000..de0b90a --- /dev/null +++ b/config/client.ssl.properties @@ -0,0 +1,9 @@ +# Keystore file +keystore.type=jks +keystore=config/client.keystore +keystorePwd=test1234 +keyPwd=test1234 + +# Truststore file +truststore=config/client.keystore +truststorePwd=test1234 \ No newline at end of file diff --git a/config/server.properties b/config/server.properties index b0e4496..e22b27d 100644 --- a/config/server.properties +++ b/config/server.properties @@ -22,7 +22,9 @@ broker.id=0 ############################# Socket Server Settings ############################# # The port the socket server listens on -port=9092 +#port=9092 + +secure.ssl.enabled=true # Hostname the broker will bind to. If not set, the server will bind to all interfaces #host.name=localhost diff --git a/config/server.ssl.properties b/config/server.ssl.properties new file mode 100644 index 0000000..001424b --- /dev/null +++ b/config/server.ssl.properties @@ -0,0 +1,32 @@ +# 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. + +#type of keystore +keystore.type=jks + +# Request client auth +want.client.auth=true + +# Require client auth +need.client.auth=true + +# Keystore file +keystore=config/server.keystore +keystorePwd=test1234 +keyPwd=test1234 + +# Truststore file +truststore=config/server.keystore +truststorePwd=test1234 diff --git a/core/src/main/scala/kafka/network/ChannelFactory.scala b/core/src/main/scala/kafka/network/ChannelFactory.scala new file mode 100644 index 0000000..2083d93 --- /dev/null +++ b/core/src/main/scala/kafka/network/ChannelFactory.scala @@ -0,0 +1,43 @@ +/** + * 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 kafka.network + +import java.nio.channels.{SocketChannel, ServerSocketChannel} + +import kafka.utils.Logging + +abstract class ChannelFactory extends Logging { + def create(serverSocketChannel: ServerSocketChannel, recvBufferSize: Int, sendBufferSize: Int): KafkaChannel = { + val channel = createSocket(serverSocketChannel) + configure(channel, recvBufferSize, sendBufferSize) + channel + } + + protected def createSocket(serverSocketChannel: ServerSocketChannel): KafkaChannel + + protected def configure(socketChannel: SocketChannel, recvBufferSize: Int, sendBufferSize: Int) { + socketChannel.configureBlocking(false) + socketChannel.socket().setTcpNoDelay(true) + socketChannel.socket().setSendBufferSize(sendBufferSize) + + debug("Accepted connection from %s on %s. sendBufferSize [actual|requested]: [%d|%d] recvBufferSize [actual|requested]: [%d|%d]" + .format(socketChannel.socket.getInetAddress, socketChannel.socket.getLocalSocketAddress, + socketChannel.socket.getSendBufferSize, sendBufferSize, + socketChannel.socket.getReceiveBufferSize, recvBufferSize)) + } +} diff --git a/core/src/main/scala/kafka/network/KafkaChannel.scala b/core/src/main/scala/kafka/network/KafkaChannel.scala new file mode 100644 index 0000000..29d9c4e --- /dev/null +++ b/core/src/main/scala/kafka/network/KafkaChannel.scala @@ -0,0 +1,54 @@ +package kafka.network + +import java.net.{SocketAddress, SocketOption, Socket} +import java.nio.ByteBuffer +import java.nio.channels.{SelectionKey, Selector, SocketChannel} +import java.util + +class KafkaChannel(val underlying: SocketChannel) extends SocketChannel(underlying.provider()) { + def shutdownInput(): SocketChannel = underlying.shutdownInput() + + def isConnectionPending: Boolean = underlying.isConnectionPending + + def socket(): Socket = underlying.socket() + + def setOption[T](name: SocketOption[T], value: T): SocketChannel = underlying.setOption(name, value) + + def write(src: ByteBuffer): Int = underlying.write(src) + + def write(srcs: Array[ByteBuffer], offset: Int, length: Int): Long = underlying.write(srcs, offset, length) + + def isConnected: Boolean = underlying.isConnected + + def getRemoteAddress: SocketAddress = underlying.getRemoteAddress + + def finishConnect(): Boolean = underlying.finishConnect() + + def read(dst: ByteBuffer): Int = underlying.read(dst) + + def read(dsts: Array[ByteBuffer], offset: Int, length: Int): Long = underlying.read(dsts, offset, length) + + def connect(remote: SocketAddress): Boolean = underlying.connect(remote) + + def bind(local: SocketAddress): SocketChannel = underlying.bind(local) + + def shutdownOutput(): SocketChannel = underlying.shutdownOutput() + + def getLocalAddress: SocketAddress = underlying.getLocalAddress + + def getOption[T](name: SocketOption[T]): T = underlying.getOption(name) + + def supportedOptions(): util.Set[SocketOption[_]] = underlying.supportedOptions() + + protected def implCloseSelectableChannel() = underlying.close() + + protected def implConfigureBlocking(block: Boolean) = underlying.configureBlocking(block) + + def forKey(key: SelectionKey, selectOption: Int = -1): KafkaChannel = this + + def isExtraReadRequired: Boolean = false + + def registerSelector(selector: Selector, opt: Int) { + underlying.register(selector, opt) + } +} diff --git a/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala b/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala new file mode 100644 index 0000000..fb605ea --- /dev/null +++ b/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala @@ -0,0 +1,27 @@ +/** + * 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 kafka.network + +import java.nio.channels.ServerSocketChannel + +class PlainSocketChannelFactory extends ChannelFactory { + + protected def createSocket(serverSocketChannel: ServerSocketChannel): KafkaChannel = { + new KafkaChannel(serverSocketChannel.accept()) + } +} diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index cee76b3..50dccd3 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -17,19 +17,19 @@ package kafka.network +import java.io._ +import java.net._ +import java.nio.channels._ import java.util import java.util.concurrent._ import java.util.concurrent.atomic._ -import java.net._ -import java.io._ -import java.nio.channels._ - -import scala.collection._ +import com.yammer.metrics.core.{Gauge, Meter} import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup import kafka.utils._ -import com.yammer.metrics.core.{Gauge, Meter} + +import scala.collection._ /** * An NIO socket server. The threading model is @@ -39,29 +39,54 @@ import com.yammer.metrics.core.{Gauge, Meter} */ class SocketServer(val brokerId: Int, val host: String, - val port: Int, + val ports: Map[Int, ChannelFactory], val numProcessorThreads: Int, val maxQueuedRequests: Int, val sendBufferSize: Int, val recvBufferSize: Int, - val maxRequestSize: Int = Int.MaxValue, - val maxConnectionsPerIp: Int = Int.MaxValue, + val maxRequestSize: Int, + val maxConnectionsPerIp: Int, val connectionsMaxIdleMs: Long, - val maxConnectionsPerIpOverrides: Map[String, Int] = Map[String, Int]()) extends Logging with KafkaMetricsGroup { - this.logIdent = "[Socket Server on Broker " + brokerId + "], " + val maxConnectionsPerIpOverrides: Map[String, Int]) extends Logging with KafkaMetricsGroup { + val port = ports.head._1 private val time = SystemTime private val processors = new Array[Processor](numProcessorThreads) @volatile private var acceptor: Acceptor = null val requestChannel = new RequestChannel(numProcessorThreads, maxQueuedRequests) + this.logIdent = "[Socket Server on Broker " + brokerId + "], " /* a meter to track the average free capacity of the network processors */ private val aggregateIdleMeter = newMeter("NetworkProcessorAvgIdlePercent", "percent", TimeUnit.NANOSECONDS) + private val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides) + + def this(brokerId: Int, + host: String, + port: Int, + numProcessorThreads: Int, + maxQueuedRequests: Int, + sendBufferSize: Int, + recvBufferSize: Int, + maxRequestSize: Int = Int.MaxValue, + maxConnectionsPerIp: Int = Int.MaxValue, + connectionsMaxIdleMs: Long, + maxConnectionsPerIpOverrides: Map[String, Int] = Map.empty[String, Int]) { + this(brokerId, + host, + Map(port -> new PlainSocketChannelFactory()), + numProcessorThreads, + maxQueuedRequests, + sendBufferSize, + recvBufferSize, + maxRequestSize, + maxConnectionsPerIp, + connectionsMaxIdleMs, + maxConnectionsPerIpOverrides) + } /** * Start the socket server */ def startup() { - val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides) for(i <- 0 until numProcessorThreads) { processors(i) = new Processor(i, time, @@ -72,7 +97,7 @@ class SocketServer(val brokerId: Int, requestChannel, quotas, connectionsMaxIdleMs) - Utils.newThread("kafka-network-thread-%d-%d".format(port, i), processors(i), false).start() + Utils.newThread("kafka-network-thread-%d".format(i), processors(i), false).start() } newGauge("ResponsesBeingSent", new Gauge[Int] { @@ -83,7 +108,7 @@ class SocketServer(val brokerId: Int, requestChannel.addResponseListener((id:Int) => processors(id).wakeup()) // start accepting connections - this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize, quotas) + this.acceptor = new Acceptor(host, ports, processors, sendBufferSize, recvBufferSize, quotas) Utils.newThread("kafka-socket-acceptor", acceptor, false).start() acceptor.awaitStartup info("Started") @@ -197,18 +222,23 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ * Thread that accepts and configures new connections. There is only need for one of these */ private[kafka] class Acceptor(val host: String, - val port: Int, + val ports: Map[Int, ChannelFactory], private val processors: Array[Processor], val sendBufferSize: Int, val recvBufferSize: Int, connectionQuotas: ConnectionQuotas) extends AbstractServerThread(connectionQuotas) { - val serverChannel = openServerSocket(host, port) + private val channelToFactory = mutable.Map.empty[SelectableChannel, ChannelFactory] /** * Accept loop that checks for new connection attempts */ def run() { - serverChannel.register(selector, SelectionKey.OP_ACCEPT); + for ((port, channelFactory) <- ports) { + val serverChannel = openServerSocket(host, port) + serverChannel.register(selector, SelectionKey.OP_ACCEPT) + channelToFactory += serverChannel -> channelFactory + } + startupComplete() var currentProcessor = 0 while(isRunning) { @@ -235,7 +265,7 @@ private[kafka] class Acceptor(val host: String, } } debug("Closing server socket and selector.") - swallowError(serverChannel.close()) + channelToFactory.foreach(entry => swallowError(entry._1.close())) swallowError(selector.close()) shutdownComplete() } @@ -267,23 +297,16 @@ private[kafka] class Acceptor(val host: String, */ def accept(key: SelectionKey, processor: Processor) { val serverSocketChannel = key.channel().asInstanceOf[ServerSocketChannel] - val socketChannel = serverSocketChannel.accept() + var socketChannel: Option[KafkaChannel] = None try { - connectionQuotas.inc(socketChannel.socket().getInetAddress) - socketChannel.configureBlocking(false) - socketChannel.socket().setTcpNoDelay(true) - socketChannel.socket().setSendBufferSize(sendBufferSize) - - debug("Accepted connection from %s on %s. sendBufferSize [actual|requested]: [%d|%d] recvBufferSize [actual|requested]: [%d|%d]" - .format(socketChannel.socket.getInetAddress, socketChannel.socket.getLocalSocketAddress, - socketChannel.socket.getSendBufferSize, sendBufferSize, - socketChannel.socket.getReceiveBufferSize, recvBufferSize)) - - processor.accept(socketChannel) + val channelFactory = channelToFactory(serverSocketChannel) + socketChannel = Some(channelFactory.create(serverSocketChannel, recvBufferSize, sendBufferSize)) + connectionQuotas.inc(socketChannel.get.socket().getInetAddress) + processor.accept(socketChannel.get) } catch { case e: TooManyConnectionsException => info("Rejected connection from %s, address already has the configured maximum of %d connections.".format(e.ip, e.count)) - close(socketChannel) + if (socketChannel.isDefined) close(socketChannel.get) } } @@ -303,7 +326,8 @@ private[kafka] class Processor(val id: Int, connectionQuotas: ConnectionQuotas, val connectionsMaxIdleMs: Long) extends AbstractServerThread(connectionQuotas) { - private val newConnections = new ConcurrentLinkedQueue[SocketChannel]() + private val newConnections = new ConcurrentLinkedQueue[KafkaChannel]() + private val regularChannelToKafkaChannel = new ConcurrentHashMap[SelectableChannel, KafkaChannel]() private val connectionsMaxIdleNanos = connectionsMaxIdleMs * 1000 * 1000 private var currentTimeNanos = SystemTime.nanoseconds private val lruConnections = new util.LinkedHashMap[SelectionKey, Long] @@ -371,6 +395,7 @@ private[kafka] class Processor(val id: Int, */ override def close(key: SelectionKey): Unit = { lruConnections.remove(key) + regularChannelToKafkaChannel.remove(key) super.close(key) } @@ -387,6 +412,7 @@ private[kafka] class Processor(val id: Int, trace("Socket server received empty response to send, registering for read: " + curr) key.interestOps(SelectionKey.OP_READ) key.attach(null) + readExtraIfNeeded(key) } case RequestChannel.SendAction => { trace("Socket server received response to send, registering for write: " + curr) @@ -414,11 +440,33 @@ private[kafka] class Processor(val id: Int, /** * Queue up a new connection for reading */ - def accept(socketChannel: SocketChannel) { + def accept(socketChannel: KafkaChannel) { newConnections.add(socketChannel) wakeup() } + private def readExtraIfNeeded(key: SelectionKey) { + try { + if (channelFor(key).isExtraReadRequired) { + read(key) + } + } catch { + case e: EOFException => { + info("Closing socket connection to %s.".format(channelFor(key).socket.getInetAddress)) + close(key) + } + case e: InvalidRequestException => { + info("Closing socket connection to %s due to invalid request: %s".format(channelFor(key).socket.getInetAddress, + e.getMessage)) + close(key) + } + case e: Throwable => { + error("Closing socket for %s because of error".format(channelFor(key).socket.getInetAddress), e) + close(key) + } + } + } + /** * Register any new connections that have been queued up */ @@ -426,7 +474,8 @@ private[kafka] class Processor(val id: Int, while(newConnections.size() > 0) { val channel = newConnections.poll() debug("Processor " + id + " listening to new connection from " + channel.socket.getRemoteSocketAddress) - channel.register(selector, SelectionKey.OP_READ) + regularChannelToKafkaChannel.put(channel.underlying, channel) + channel.registerSelector(selector, SelectionKey.OP_READ) } } @@ -435,7 +484,8 @@ private[kafka] class Processor(val id: Int, */ def read(key: SelectionKey) { lruConnections.put(key, currentTimeNanos) - val socketChannel = channelFor(key) + val socketChannel = channelFor(key, SelectionKey.OP_READ) + if (socketChannel == null) return var receive = key.attachment.asInstanceOf[Receive] if(key.attachment == null) { receive = new BoundedByteBufferReceive(maxRequestSize) @@ -464,7 +514,8 @@ private[kafka] class Processor(val id: Int, * Process writes to ready sockets */ def write(key: SelectionKey) { - val socketChannel = channelFor(key) + val socketChannel = channelFor(key, SelectionKey.OP_WRITE) + if (socketChannel == null) return val response = key.attachment().asInstanceOf[RequestChannel.Response] val responseSend = response.responseSend if(responseSend == null) @@ -476,6 +527,7 @@ private[kafka] class Processor(val id: Int, key.attach(null) trace("Finished writing, registering for read on connection " + socketChannel.socket.getRemoteSocketAddress()) key.interestOps(SelectionKey.OP_READ) + readExtraIfNeeded(key) } else { trace("Did not finish writing, registering for write again on connection " + socketChannel.socket.getRemoteSocketAddress()) key.interestOps(SelectionKey.OP_WRITE) @@ -483,7 +535,11 @@ private[kafka] class Processor(val id: Int, } } - private def channelFor(key: SelectionKey) = key.channel().asInstanceOf[SocketChannel] + private def channelFor(key: SelectionKey, selectOption: Int = -1) = kafkaChannel(key.channel()).forKey(key, selectOption) + + private def kafkaChannel(regularChannel: SelectableChannel): KafkaChannel = { + regularChannelToKafkaChannel.get(regularChannel) + } private def maybeCloseOldestConnection { if(currentTimeNanos > nextIdleCloseCheckTime) { diff --git a/core/src/main/scala/kafka/network/ssl/KeyStores.scala b/core/src/main/scala/kafka/network/ssl/KeyStores.scala new file mode 100644 index 0000000..6699ba5 --- /dev/null +++ b/core/src/main/scala/kafka/network/ssl/KeyStores.scala @@ -0,0 +1,35 @@ +/** + * 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 kafka.network.ssl + +import javax.net.ssl.SSLContext + +import kafka.network.ssl.store.JKSInitializer + +trait StoreInitializer { + def initialize(config: SSLConnectionConfig): SSLContext +} + +object KeyStores { + def getKeyStore(name: String): StoreInitializer = { + name.toLowerCase match { + case JKSInitializer.name => JKSInitializer + case _ => throw new RuntimeException("%s is an unknown key store".format(name)) + } + } +} diff --git a/core/src/main/scala/kafka/network/ssl/SSLAuth.scala b/core/src/main/scala/kafka/network/ssl/SSLAuth.scala new file mode 100644 index 0000000..dea6843 --- /dev/null +++ b/core/src/main/scala/kafka/network/ssl/SSLAuth.scala @@ -0,0 +1,53 @@ +/** + * 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 kafka.network.ssl + +import java.util.concurrent.atomic.AtomicBoolean +import javax.net.ssl.SSLContext + +import kafka.utils.Logging + +object SSLAuth extends Logging { + private val initialized = new AtomicBoolean(false) + private var authContext: SSLContext = null + + def isInitialized = initialized.get + + def sslContext = { + if (!initialized.get) { + throw new IllegalStateException("Secure authentication is not initialized.") + } + authContext + } + + def initialize(config: SSLConnectionConfig) { + if (initialized.get) { + warn("Attempt to reinitialize auth context") + return + } + + info("Initializing secure authentication") + + val initializer = KeyStores.getKeyStore(config.keystoreType) + authContext = initializer.initialize(config) + + initialized.set(true) + + info("Secure authentication initialization has been successfully completed") + } +} diff --git a/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala b/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala new file mode 100644 index 0000000..6261607 --- /dev/null +++ b/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala @@ -0,0 +1,31 @@ +/** + * 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 kafka.network.ssl + +import java.nio.channels.ServerSocketChannel + +import kafka.network.{ChannelFactory, KafkaChannel} + +class SSLChannelFactory(wantClientAuth: Boolean, + needClientAuth: Boolean) extends ChannelFactory { + protected def createSocket(serverSocketChannel: ServerSocketChannel): KafkaChannel = { + SSLSocketChannel.makeSecureServerConnection(serverSocketChannel.accept(), + wantClientAuth, + needClientAuth) + } +} diff --git a/core/src/main/scala/kafka/network/ssl/SSLConnectionConfig.scala b/core/src/main/scala/kafka/network/ssl/SSLConnectionConfig.scala new file mode 100644 index 0000000..32d4361 --- /dev/null +++ b/core/src/main/scala/kafka/network/ssl/SSLConnectionConfig.scala @@ -0,0 +1,57 @@ +/** + * 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 kafka.network.ssl + +import kafka.network.ConnectionConfig +import kafka.utils.{Utils, VerifiableProperties} + +object SSLConnectionConfig { + lazy val server = new SSLConnectionConfig("config/server.ssl.properties") + lazy val client = new SSLConnectionConfig("config/client.ssl.properties") +} + +class SSLConnectionConfig(path: String) extends ConnectionConfig { + val props = new VerifiableProperties(Utils.loadProps(path)) + + val host: String = props.getString("host", "") + + val port: Int = props.getInt("port", 9093) + + /** Keystore file location */ + val keystore = props.getString("keystore") + + /** Keystore file password */ + val keystorePwd = props.getString("keystorePwd") + + /** Keystore key password */ + val keyPwd = props.getString("keyPwd") + + /** Truststore file location */ + val truststore = props.getString("truststore") + + /** Truststore file password */ + val truststorePwd = props.getString("truststorePwd") + + val keystoreType = props.getString("keystore.type") + + /** Request client auth */ + val wantClientAuth = props.getBoolean("want.client.auth", false) + + /** Require client auth */ + val needClientAuth = props.getBoolean("need.client.auth", false) +} diff --git a/core/src/main/scala/kafka/network/ssl/SSLSocketChannel.scala b/core/src/main/scala/kafka/network/ssl/SSLSocketChannel.scala new file mode 100644 index 0000000..6ec9f36 --- /dev/null +++ b/core/src/main/scala/kafka/network/ssl/SSLSocketChannel.scala @@ -0,0 +1,661 @@ +/** + * 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 kafka.network.ssl + +import java.io.IOException +import java.net._ +import java.nio.ByteBuffer +import java.nio.channels._ +import javax.net.ssl._ +import javax.net.ssl.SSLEngineResult._ +import java.util.concurrent._ +import java.util.concurrent.atomic.AtomicInteger +import kafka.network.KafkaChannel +import kafka.utils.Logging +import java.util + +object SSLSocketChannel { + + def makeSecureClientConnection(sch: SocketChannel, host: String, port: Int) = { + val engine = SSLAuth.sslContext.createSSLEngine(host, port) + engine.setEnabledProtocols(Array("SSLv3")) + engine.setUseClientMode(true) + new SSLSocketChannel(sch, engine) + } + + def makeSecureServerConnection(socketChannel: SocketChannel, + wantClientAuth: Boolean = true, + needClientAuth: Boolean = true) = { + val engine = socketChannel.socket.getRemoteSocketAddress match { + case ise: InetSocketAddress => + SSLAuth.sslContext.createSSLEngine(ise.getHostName, ise.getPort) + case _ => + SSLAuth.sslContext.createSSLEngine() + } + engine.setEnabledProtocols(Array("SSLv3")) + engine.setUseClientMode(false) + if (wantClientAuth) { + engine.setWantClientAuth(true) + } + if (needClientAuth) { + engine.setNeedClientAuth(true) + } + new SSLSocketChannel(socketChannel, engine) + } + + val simulateSlowNetwork = false + + val runningTasks = -2 + private[this] lazy val counter = new AtomicInteger(0) + private[kafka] lazy val executor = new ThreadPoolExecutor(2, 10, + 60L, TimeUnit.SECONDS, + new SynchronousQueue[Runnable](), + new ThreadFactory() { + override def newThread(r: Runnable): Thread = { + val thread = new Thread(r, "SSLSession-Task-Thread-%d".format(counter.incrementAndGet())) + thread.setDaemon(true) + thread + } + }) +} + +class SSLSocketChannel(val socketChannel: SocketChannel, val sslEngine: SSLEngine) + extends KafkaChannel(socketChannel) with Logging { + + import SSLSocketChannel.executor + + private[this] class SSLTasker(val runnable: Runnable) extends Runnable { + selectionKey.interestOps(0) + + override def run(): Unit = { + try { + runnable.run() + outer.synchronized { + handshakeStatus = sslEngine.getHandshakeStatus + handshakeStatus match { + case HandshakeStatus.NEED_WRAP => + debug("sslTasker setting up to write for %s".format(socketChannel.socket.getRemoteSocketAddress)) + selectionKey.interestOps(SelectionKey.OP_WRITE) + case HandshakeStatus.NEED_UNWRAP => + if (peerNetData.position > 0) { + debug("sslTasker found existing data %s. running hanshake for %s".format(peerNetData, + socketChannel.socket.getRemoteSocketAddress)) + val init = outer.handshake(SelectionKey.OP_READ, selectionKey) + if (init == 0) { + debug("sslTasker setting up to read after hanshake") + selectionKey.interestOps(SelectionKey.OP_READ) + } else if (init != SSLSocketChannel.runningTasks) { + debug("sslTasker setting up for operation %d after hanshake for %s".format(init, + socketChannel.socket.getRemoteSocketAddress)) + selectionKey.interestOps(init) + } + } else { + debug("sslTasker setting up to read for %s".format(socketChannel.socket.getRemoteSocketAddress)) + selectionKey.interestOps(SelectionKey.OP_READ) + } + case HandshakeStatus.NEED_TASK => + val runnable = sslEngine.getDelegatedTask + if (runnable != null) { + debug("sslTasker running next task for %s".format(socketChannel.socket.getRemoteSocketAddress)) + executor.execute(new SSLTasker(runnable)) + handshakeStatus = null + } + return + case _ => + throw new SSLException("unexpected handshakeStatus: " + handshakeStatus) + } + selectionKey.selector.wakeup() + } + } catch { + case t: Throwable => + error("Unexpected exception", t) + } + } + } + + private[this] val outer = this + + /** + * The engine handshake status. + */ + private[this] var handshakeStatus: HandshakeStatus = HandshakeStatus.NOT_HANDSHAKING + + /** + * The initial handshake ops. + */ + @volatile private[this] var initialized = -1 + + /** + * Marker for shutdown status + */ + private[this] var shutdown = false + + private[this] var peerAppData = ByteBuffer.allocate(sslEngine.getSession.getApplicationBufferSize) + private[this] var myNetData = ByteBuffer.allocate(sslEngine.getSession.getPacketBufferSize) + private[this] var peerNetData = ByteBuffer.allocate(sslEngine.getSession.getPacketBufferSize) + private[this] val emptyBuffer = ByteBuffer.allocate(0) + + myNetData.limit(0) + + socketChannel.configureBlocking(false) + + private[this] var blocking = false + private[this] lazy val blockingSelector = Selector.open() + private[this] var blockingKey: SelectionKey = null + + @volatile private[this] var selectionKey: SelectionKey = null + + def simulateBlocking(b: Boolean) = { + blocking = b + } + + override def connect(remote: SocketAddress): Boolean = { + debug("SSLSocketChannel Connecting to Remote : " + remote) + val ret = socketChannel.connect(remote) + if (blocking) { + while (!finishConnect()) { + try { + Thread.sleep(10) + } catch { + case _: InterruptedException => + } + } + blockingKey = socketChannel.register(blockingSelector, SelectionKey.OP_READ) + handshakeInBlockMode(SelectionKey.OP_WRITE) + true + } else ret + } + + def isReadable = finished && (peerAppData.position > 0 || peerNetData.position > 0) + + override def read(dst: ByteBuffer): Int = { + this.synchronized { + if (peerAppData.position >= dst.remaining) { + return readFromPeerData(dst) + } else if (socketChannel.socket.isInputShutdown) { + throw new ClosedChannelException + } else if (initialized != 0) { + handshake(SelectionKey.OP_READ, selectionKey) + return 0 + } else if (shutdown) { + shutdown() + return -1 + } else if (sslEngine.isInboundDone) { + return -1 + } else { + val count = readRaw() + if (count <= 0 && peerNetData.position == 0) return count.asInstanceOf[Int] + } + + if (unwrap(false) < 0) return -1 + + readFromPeerData(dst) + } + } + + override def read(destination: Array[ByteBuffer], offset: Int, length: Int): Long = { + var n = 0 + var i = offset + def localReadLoop() { + while (i < length) { + if (destination(i).hasRemaining) { + val x = read(destination(i)) + if (x > 0) { + n += x + if (!destination(i).hasRemaining) { + return + } + } else { + if ((x < 0) && (n == 0)) { + n = -1 + } + return + } + } + i = i + 1 + } + } + localReadLoop() + n + } + + override def write(source: ByteBuffer): Int = { + this.synchronized { + if (myNetData.hasRemaining) { + writeRaw(myNetData) + return 0 + } else if (socketChannel.socket.isOutputShutdown) { + throw new ClosedChannelException + } else if (initialized != 0) { + handshake(SelectionKey.OP_WRITE, selectionKey) + return 0 + } else if (shutdown) { + shutdown() + return -1 + } + + val written = wrap(source) + + while (myNetData.hasRemaining) + writeRaw(myNetData) + written + } + } + + override def write(sources: Array[ByteBuffer], offset: Int, length: Int): Long = { + var n = 0 + var i = offset + def localWriteLoop { + while (i < length) { + if (sources(i).hasRemaining) { + var x = write(sources(i)) + if (x > 0) { + n += x + if (!sources(i).hasRemaining) { + return + } + } else { + return + } + } + i = i + 1 + } + } + localWriteLoop + n + } + + def finished(): Boolean = initialized == 0 + + override def toString = "SSLSocketChannel[" + socketChannel.toString + "]" + + override protected def implCloseSelectableChannel(): Unit = { + try { + _shutdown() + } catch { + case x: Exception => + } + socketChannel.close() + } + + override protected def implConfigureBlocking(block: Boolean) { + simulateBlocking(block) + if (!block) socketChannel.configureBlocking(block) + } + + def handshake(o: Int, key: SelectionKey): Int = { + def writeIfReadyAndNeeded(mustWrite: Boolean): Boolean = { + if ((o & SelectionKey.OP_WRITE) != 0) { + writeRaw(myNetData) + myNetData.remaining > 0 + } else mustWrite + } + def readIfReadyAndNeeded(mustRead: Boolean): Boolean = { + if ((o & SelectionKey.OP_READ) != 0) { + if (readRaw() < 0) { + shutdown = true + socketChannel.close() + return true + } + val oldPos = peerNetData.position + unwrap(true) + oldPos == peerNetData.position + } else mustRead + } + + def localHandshake(): Int = { + while (true) { + handshakeStatus match { + case HandshakeStatus.NOT_HANDSHAKING => + info("begin ssl handshake for %s/%s".format(socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + sslEngine.beginHandshake() + handshakeStatus = sslEngine.getHandshakeStatus + case HandshakeStatus.NEED_UNWRAP => + debug("need unwrap in ssl handshake for %s/%s".format(socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + if (readIfReadyAndNeeded(true) && handshakeStatus != HandshakeStatus.FINISHED) { + debug("select to read more for %s/%s".format(socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + return SelectionKey.OP_READ + } + case HandshakeStatus.NEED_WRAP => + debug("need wrap in ssl handshake for %s/%s".format(socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + if (myNetData.remaining == 0) { + wrap(emptyBuffer) + } + if (writeIfReadyAndNeeded(true)) { + debug("select to write more for %s/%s".format(socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + return SelectionKey.OP_WRITE + } + case HandshakeStatus.NEED_TASK => + handshakeStatus = runTasks() + case HandshakeStatus.FINISHED => + info("finished ssl handshake for %s/%s".format(socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + return 0 + case null => + return SSLSocketChannel.runningTasks + } + } + o + } + + this.synchronized { + if (initialized == 0) return initialized + + if (selectionKey == null) selectionKey = key + + if (initialized != -1) { + if (writeIfReadyAndNeeded(false)) return o + } + val init = localHandshake() + if (init != SSLSocketChannel.runningTasks) { + initialized = init + } + init + } + } + + def shutdown() { + debug("SSLSocketChannel shutting down with locking") + this.synchronized(_shutdown()) + socketChannel.close() + } + + private def _shutdown() { + debug("SSLSocketChannel shutting down with out locking") + shutdown = true + + try { + if (!sslEngine.isOutboundDone) sslEngine.closeOutbound() + + myNetData.compact() + while (!sslEngine.isOutboundDone) { + val res = sslEngine.wrap(emptyBuffer, myNetData) + if (res.getStatus != Status.CLOSED) { + throw new SSLException("Unexpected shutdown status '%s'".format(res.getStatus)) + } + + myNetData.flip() + try { + while (myNetData.hasRemaining) + writeRaw(myNetData) + } catch { + case ignore: IOException => + } + } + } finally { + if (blockingKey != null) { + try { + blockingKey.cancel() + } finally { + blockingKey = null + blockingSelector.close() + } + } + } + } + + private def handshakeInBlockMode(ops: Int) = { + var o = ops + while (o != 0) { + val tops = handshake(o, null) + if (tops == o) { + try { + Thread.sleep(10) + } catch { + case _: InterruptedException => + } + } else { + o = tops + } + } + o + } + + private[this] def readRaw(): Long = { + def blockIfNeeded() { + if (blockingKey != null) { + try { + blockingSelector.select(5000) + } catch { + case t: Throwable => error("Unexpected error in blocking select", t) + } + } + } + this.synchronized { + blockIfNeeded() + try { + val n = socketChannel.read(peerNetData) + if (n < 0) { + sslEngine.closeInbound() + } + n + } catch { + case x: IOException => + sslEngine.closeInbound() + throw x + } + } + } + + private[this] def unwrap(isHandshaking: Boolean): Int = { + val pos = peerAppData.position + peerNetData.flip() + trace("unwrap: flipped peerNetData %s for %s/%s".format(peerNetData, + socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + try { + while (peerNetData.hasRemaining) { + val result = sslEngine.unwrap(peerNetData, peerAppData) + handshakeStatus = result.getHandshakeStatus + result.getStatus match { + case SSLEngineResult.Status.OK => + if (handshakeStatus == HandshakeStatus.NEED_TASK) { + handshakeStatus = runTasks() + if (handshakeStatus == null) return 0 + } + if (isHandshaking && handshakeStatus == HandshakeStatus.FINISHED) { + return peerAppData.position - pos + } + case SSLEngineResult.Status.BUFFER_OVERFLOW => + peerAppData = expand(peerAppData, sslEngine.getSession.getApplicationBufferSize) + case SSLEngineResult.Status.BUFFER_UNDERFLOW => + return 0 + case SSLEngineResult.Status.CLOSED => + if (peerAppData.position == 0) { + trace("uwrap: shutdown for %s/%s".format(peerAppData, + socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + shutdown() + return -1 + } else { + trace("uwrap: shutdown with non-empty peerAppData %s for %s/%s".format(peerAppData, + socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + shutdown = true + return 0 + } + case _ => + throw new SSLException("Unexpected state!") + } + } + } finally { + peerNetData.compact() + trace("unwrap: compacted peerNetData %s for %s/%s".format(peerNetData, + socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + } + peerAppData.position - pos + } + + private[this] def wrap(src: ByteBuffer): Int = { + val written = src.remaining + myNetData.compact() + trace("wrap: compacted myNetData %s for %s/%s".format(myNetData, + socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + try { + do { + val result = sslEngine.wrap(src, myNetData) + handshakeStatus = result.getHandshakeStatus + result.getStatus match { + case SSLEngineResult.Status.OK => + if (handshakeStatus == HandshakeStatus.NEED_TASK) { + handshakeStatus = runTasks() + if (handshakeStatus == null) return 0 + } + case SSLEngineResult.Status.BUFFER_OVERFLOW => + val size = if (src.remaining * 2 > sslEngine.getSession.getApplicationBufferSize) src.remaining * 2 + else sslEngine.getSession.getApplicationBufferSize + myNetData = expand(myNetData, size) + case SSLEngineResult.Status.CLOSED => + shutdown() + throw new IOException("Write error received Status.CLOSED") + case _ => + throw new SSLException("Unexpected state!") + } + } while (src.hasRemaining) + } finally { + myNetData.flip() + trace("wrap: flipped myNetData %s for %s/%s".format(myNetData, + socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + } + written + } + + private[this] def writeRaw(out: ByteBuffer): Long = { + def writeTwo(i: ByteBuffer): ByteBuffer = { + val o = ByteBuffer.allocate(2) + var rem = i.limit - i.position + if (rem > o.capacity) rem = o.capacity + var c = 0 + while (c < rem) { + o.put(i.get) + c += 1 + } + o.flip() + o + } + try { + if (out.hasRemaining) { + socketChannel.write(if (SSLSocketChannel.simulateSlowNetwork) writeTwo(out) else out) + } else 0 + } catch { + case x: IOException => + sslEngine.closeOutbound() + shutdown = true + throw x + } + } + + private[this] def runTasks(ops: Int = SelectionKey.OP_READ): HandshakeStatus = { + val reInitialize = initialized match { + case 0 => + initialized = ops + info("runTasks running renegotiation for %s/%s".format(socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + true + case _ => false + } + var runnable: Runnable = sslEngine.getDelegatedTask + if (!blocking && selectionKey != null) { + debug("runTasks asynchronously in ssl handshake for %s/%s".format(socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + if (runnable != null) { + executor.execute(new SSLTasker(runnable)) + } + null + } else { + debug("runTasks synchronously in ssl handshake for %s/%s".format(socketChannel.socket.getRemoteSocketAddress, + socketChannel.socket.getLocalSocketAddress)) + while (runnable != null) { + runnable.run() + runnable = sslEngine.getDelegatedTask + } + if (reInitialize) { + handshakeInBlockMode(ops) + } + sslEngine.getHandshakeStatus + } + } + + private[this] def expand(src: ByteBuffer, ensureSize: Int): ByteBuffer = { + if (src.remaining < ensureSize) { + val newBuffer = ByteBuffer.allocate(src.capacity + ensureSize) + if (src.position > 0) { + src.flip() + newBuffer.put(src) + } + newBuffer + } else { + src + } + } + + private[this] def readFromPeerData(dest: ByteBuffer): Int = { + peerAppData.flip() + try { + var remaining = peerAppData.remaining + if (remaining > 0) { + if (remaining > dest.remaining) { + remaining = dest.remaining + } + var i = 0 + while (i < remaining) { + dest.put(peerAppData.get) + i = i + 1 + } + } + remaining + } finally { + peerAppData.compact() + } + } + + override def forKey(key: SelectionKey, selectOption: Int = -1): KafkaChannel = { + if (selectOption >= 0 && !finished()) { + var done = false + try { + val next = handshake(key.interestOps(), key) + if (next == 0) { + // when handshake is complete and we are doing a read so ahead with the read + // otherwise go back to read mode + if (selectOption == SelectionKey.OP_READ) { + done = true + } else { + key.interestOps(SelectionKey.OP_READ) + } + } else if (next != SSLSocketChannel.runningTasks) { + key.interestOps(next) + } + } catch { + case e: SSLException => // just ignore SSL disconnect errors + debug("SSLException: " + e) + throw new IllegalStateException(e) + } + if (done) this else null + } else this + } + + override def isExtraReadRequired = isReadable +} diff --git a/core/src/main/scala/kafka/network/ssl/store/JKSInitializer.scala b/core/src/main/scala/kafka/network/ssl/store/JKSInitializer.scala new file mode 100644 index 0000000..0e91b5e --- /dev/null +++ b/core/src/main/scala/kafka/network/ssl/store/JKSInitializer.scala @@ -0,0 +1,62 @@ +/** + * 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 kafka.network.ssl.store + +import java.io.FileInputStream +import javax.net.ssl._ + +import kafka.network.ssl.{SSLConnectionConfig, StoreInitializer} + +object JKSInitializer extends StoreInitializer { + val name = "jks" + + def initialize(config: SSLConnectionConfig) = { + val tms = config.truststorePwd match { + case pw: String => + val ts = java.security.KeyStore.getInstance("JKS") + val fis: FileInputStream = new FileInputStream(config.truststore) + ts.load(fis, pw.toCharArray) + fis.close() + + val tmf = TrustManagerFactory.getInstance("SunX509") + tmf.init(ts) + tmf.getTrustManagers + case _ => null + } + val kms = config.keystorePwd match { + case pw: String => + val ks = java.security.KeyStore.getInstance("JKS") + val fis: FileInputStream = new FileInputStream(config.keystore) + ks.load(fis, pw.toCharArray) + fis.close() + + val kmf = KeyManagerFactory.getInstance("SunX509") + kmf.init(ks, if (config.keyPwd != null) config.keyPwd.toCharArray else pw.toCharArray) + kmf.getKeyManagers + case _ => null + } + + initContext(tms, kms) + } + + private def initContext(tms: Array[TrustManager], kms: Array[KeyManager]): SSLContext = { + val authContext = SSLContext.getInstance("TLS") + authContext.init(kms, tms, null) + authContext + } +} diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 165c816..01667e6 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -18,9 +18,10 @@ package kafka.server import java.util.Properties -import kafka.message.{MessageSet, Message} + import kafka.consumer.ConsumerConfig -import kafka.utils.{VerifiableProperties, ZKConfig, Utils} +import kafka.message.{Message, MessageSet} +import kafka.utils.{Utils, VerifiableProperties, ZKConfig} /** * Configuration settings for the kafka server @@ -81,7 +82,10 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /*********** Socket Server Configuration ***********/ /* the port to listen and accept connections on */ - val port: Int = props.getInt("port", 6667) + val port: Int = props.getInt("port", -1) + + /* is this running SSL */ + val sslEnabled: Boolean = props.getBoolean("secure.ssl.enabled", false) /* hostname of broker. If this is set, it will only bind to this address. If this is not set, * it will bind to all interfaces */ @@ -319,5 +323,5 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off */ val deleteTopicEnable = props.getBoolean("delete.topic.enable", false) - + } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 3e9e91f..f1b15fe 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -17,23 +17,24 @@ package kafka.server -import kafka.admin._ -import kafka.log.LogConfig -import kafka.log.CleanerConfig -import kafka.log.LogManager -import kafka.utils._ -import java.util.concurrent._ -import atomic.{AtomicInteger, AtomicBoolean} import java.io.File -import java.net.BindException -import org.I0Itec.zkclient.ZkClient -import kafka.controller.{ControllerStats, KafkaController} +import java.util.concurrent._ +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} + +import com.yammer.metrics.core.Gauge +import kafka.admin._ +import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse} import kafka.cluster.Broker -import kafka.api.{ControlledShutdownResponse, ControlledShutdownRequest} import kafka.common.ErrorMapping -import kafka.network.{Receive, BlockingChannel, SocketServer} +import kafka.controller.{ControllerStats, KafkaController} +import kafka.log.{CleanerConfig, LogConfig, LogManager} import kafka.metrics.KafkaMetricsGroup -import com.yammer.metrics.core.Gauge +import kafka.network.ssl.{SSLChannelFactory, SSLConnectionConfig, SSLAuth} +import kafka.network._ +import kafka.utils._ +import org.I0Itec.zkclient.ZkClient + +import scala.collection.{Map, mutable} /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required @@ -88,14 +89,15 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg socketServer = new SocketServer(config.brokerId, config.hostName, - config.port, + portsToChannelFactories, config.numNetworkThreads, config.queuedMaxRequests, config.socketSendBufferBytes, config.socketReceiveBufferBytes, config.socketRequestMaxBytes, config.maxConnectionsPerIp, - config.connectionsMaxIdleMs) + config.connectionsMaxIdleMs, + Map.empty[String, Int]) socketServer.startup() replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown) @@ -135,6 +137,19 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg throw e } } + + def portsToChannelFactories: Map[Int, ChannelFactory] = { + val portToChannelFactory = mutable.Map.empty[Int, ChannelFactory] + if (config.port > 0) { + portToChannelFactory += config.port -> new PlainSocketChannelFactory() + } + if (config.sslEnabled) { + val sslConfig = SSLConnectionConfig.server + SSLAuth.initialize(sslConfig) + portToChannelFactory += sslConfig.port -> new SSLChannelFactory(sslConfig.wantClientAuth, sslConfig.needClientAuth) + } + portToChannelFactory.toMap + } private def initZk(): ZkClient = { info("Connecting to zookeeper on " + config.zkConnect) -- 1.8.4.msysgit.0 From 760a74d836beca3056aa0aec342d2c243226e6b9 Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Fri, 17 Oct 2014 17:12:17 +0300 Subject: [PATCH 02/12] working --- config/client.keystore | Bin 0 -> 2074 bytes config/client.public-key | 14 ++++++++++++++ config/server.keystore | Bin 0 -> 2073 bytes config/server.public-key | 14 ++++++++++++++ core/src/main/scala/kafka/server/KafkaServer.scala | 3 +++ 5 files changed, 31 insertions(+) create mode 100644 config/client.keystore create mode 100644 config/client.public-key create mode 100644 config/server.keystore create mode 100644 config/server.public-key diff --git a/config/client.keystore b/config/client.keystore new file mode 100644 index 0000000000000000000000000000000000000000..8ea0757e359f2c2af5bac9222f31eaf771d4f226 GIT binary patch literal 2074 zcmezO_TO6u1_mZLW-CrDDoZV5U|_V@)e_ynz#5@vYG4UeQESk|RB6z}_-Fw$6C)E7 zOW@XSNdsOsPOUbNw(q=*jI68-28|7d+ym@ni?5K zfw`6j7EtbBaAMf); zKkMbY{&~R$WfjHP9b#68xV)cUx>EgFqOUyY)w7~1nVY9ndz+#I*{@x6DXf3JB=TNv za{lh;XYVpGGcqtDI}aGG%s_Ybv9An%D>Ut$x5uf!OdP$cYoA)wmY+`1IW*yn-fe+s z)!*jF*Bxr|;{Tx(>N>gP*n>3nGQ;(+O=1k6Hu*cu*`LN=Gv`11kE42N6?r~C()TQV z^vY7eM`5DS_0#J`7pBNBHgak2uM@VITC}`x%Om6N?OV%y62y*pv%J{CW{fFicSRdNEaI}!J-lzrMSgLstdnl>?Y`{J|2Ayni>bP#Xz91`!L9XG z&sF`-wXOZ}@4bb+hTm5j}$r{e48-g*D? zZIk5^fps(fXL3x=EV^7??s4Pb6~=}s)+XhK*A`Yf&+k8aHu}SAp2RC68(x$LoY0J$ z8Fr%Dj_3nbr;Dm32+X-yTP*jiTIsp{ae?*hdZ=Ls|tNd{opV? z+|*>>;sbBxAMd@n)OTHC9dbmx%V?nS#bZZ67S}3fA6hV`rC8mnatipYq_8F zZsl?M#UCOc&wlTz{4!zrJQ=yIy&F%@2wrpS+zqL>d`pg5U7gC+T9@^C$20v^pKb0o zFtL9t7Yz7V)OXjg^SyCJ&e{L=#Yzm#JVr+~l9z}rTRBzqsU81kIe)k2hv#En9h}jq z5h=dYWsYs)+Shzr{ay;_RDD^s`AE1|fS(B@MdK-5Bhj+qfEHcOIhm<>B`Bq01|;3p-2{Z4s|SmNlNsDCPo6x`=z66)|K#EYYRR@dxdPQwYZop(C%fju(Md&5 zQB w{r^iyzu9Nq)y9kZ^^P|>zx}+B*J->@%B|w_rW5-sPy2d$O-uT8r(w<>02PHJ!vFvP literal 0 HcmV?d00001 diff --git a/config/client.public-key b/config/client.public-key new file mode 100644 index 0000000..8b10172 --- /dev/null +++ b/config/client.public-key @@ -0,0 +1,14 @@ +-----BEGIN CERTIFICATE----- +MIICeTCCAeKgAwIBAgIEUbWLWTANBgkqhkiG9w0BAQUFADCBgDELMAkGA1UEBhMCVVMxEzARBgNV +BAgTCkNhbGlmb3JuaWExFjAUBgNVBAcTDVNhbiBGcmFuY2lzY28xFzAVBgNVBAoTDlNhbGVzZm9y +Y2UuY29tMRQwEgYDVQQLEwtBcHBsaWNhdGlvbjEVMBMGA1UEAxMMS2Fma2EgQ2xpZW50MB4XDTEz +MDYxMDA4MTYyNVoXDTEzMDkwODA4MTYyNVowgYAxCzAJBgNVBAYTAlVTMRMwEQYDVQQIEwpDYWxp +Zm9ybmlhMRYwFAYDVQQHEw1TYW4gRnJhbmNpc2NvMRcwFQYDVQQKEw5TYWxlc2ZvcmNlLmNvbTEU +MBIGA1UECxMLQXBwbGljYXRpb24xFTATBgNVBAMTDEthZmthIENsaWVudDCBnzANBgkqhkiG9w0B +AQEFAAOBjQAwgYkCgYEAqtviDb8lrz+gfz91B1CXtaF3E0CRUh3YeHx1AwVqX8sXvTviAc6qM2Sv +Cpwi0x+dbq09uTrgo1NjAEc5ycnAgUXUi2/Jo6AmYz0MbRB7lX2hpc4drPDFknLlZCI1hgat42N4 +dq0L0fJ30VHnpvBErtnHij8SGBX55bKTu0PVPy8CAwEAATANBgkqhkiG9w0BAQUFAAOBgQBDaeOz +RbNdFy8d2eafnUEv2xt7/zTitTADrbs9RD3ZqTD8oziHNwQXFPuEtknx8myIZfephjjB0jHuGdrV +/xSAQwnufZQXJbfYpouKrF1mTT7Myn+kl6nGETu4tZyVH0MQqMFPhV5x6h5o9f/6Ei+DTK7VgdEv +f0HYifb50G6JMw== +-----END CERTIFICATE----- diff --git a/config/server.keystore b/config/server.keystore new file mode 100644 index 0000000000000000000000000000000000000000..fe0dd6a4803060a9c9be80b5d4649ca4e579c13b GIT binary patch literal 2073 zcmezO_TO6u1_mZLW=qb=OwB7{U|_V@)jWQafi*(U)W8y`qSl~^snVc{@zDZiCPpSE zmcXsukp{eMoLX%jZQpqr8Ch8w3>q5@xeYkkm_u3EgqcEv4TTK^K^zWYF6YFY%(VQX zyv#&HF#{2hAiFScaAKZ#6Son z!7a?~SWu9YnVeXXnV)AUY9I^}Wftb~PE5;ARB#5n+dxj7*U;F&%+SEV!qCjfGz!eM zG_Zhj2ZIwE=Oc$3BP#=QV=pl5I++?98CKnX#JgX0z5RlE`%?CR>01|;3p-2{Z4s|SmNlNsDCPo6x`=z66)|K#EYYRR@d zxdPQwYZop(C%fju(Md&5Q`?sCq2yxDbgthm1H&1duHI_lq+uKsWGXsZG9+TFG;wl`NA z{8?<#Zq6bu^1Ef5=f_Vu9jV_}wpkp!WcW_<*46(a4bGhJYNv>+ZojdtyK7Bsny=lN zQ}s)xuRJDby<_W~sq)SOD-Qa%#udJj%lP{Lmymw5&$_FP7xn8MZ*+e9c_FXU7??g8 z8CY{Gvr{XXzWATD>)7V)6LG%pc5Pc>eZ=`!G5oG(Kl4ws-_-J7=oXoh zboln{6+Op-?p@fsQ*I^4fymG8rx~U#ND>YgRr=o=Ei9-jqu|}~?tRw|)7wAY)eDb_&JaKlU z&ZON>+z)>%a%`L;D=YP0F-xs(e)TVlE^QkT=zK>ew*>B+q?LU5=HSg1#9?wsQG_LT=uX*!damGXUmnu$K z&kma}4JeV7So_+oKl;v(`~^E?APE{z;VOxi3kS61aw^KtPAx(yTuqG(kqcK4cQ7~+ zU*S6Ek(bcs36_6sJvY4!+>krhw^r?J)TMxcYmJ+~I`VsQ&Em4LZdXoPy7{kI4*RKD z&fgul(=`;ltj-lbZF?)&aIu|TM{VgJ&I!fOdikz@Ua&z~MKN}VnAIUJ@28ipRDYJ} zD-U}0tf)%n<|)t8R4yqBAtzx(;wySNJ1KK7NtZ-u74^Y%FPmx-fS zb?sA&+VayWI)^5l(Yq}Wt@_*i__{+)Ui?3lLR}}99D9(aUS_!dwMmTO(9oNBW+nk6u{{_$W*ix_)}S=)x5F#YQge{dK|?Q;U}OZFywey?tw$ oPlDJHZ new SSLChannelFactory(sslConfig.wantClientAuth, sslConfig.needClientAuth) } + + if (portToChannelFactory.isEmpty) portToChannelFactory += 9092 -> new PlainSocketChannelFactory() + portToChannelFactory.toMap } -- 1.8.4.msysgit.0 From e5ee58e22713aa59845469c765dd6f56ad62eddb Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Fri, 17 Oct 2014 19:56:28 +0300 Subject: [PATCH 03/12] working --- .../java/org/apache/kafka/common/utils/Utils.java | 18 +++++++++++---- core/src/main/scala/kafka/client/ClientUtils.scala | 8 +++---- core/src/main/scala/kafka/cluster/Broker.scala | 20 ++++++++++------- .../main/scala/kafka/network/ConnectionType.scala | 23 +++++++++++++++++++ .../main/scala/kafka/server/KafkaHealthcheck.scala | 8 ++++--- core/src/main/scala/kafka/utils/ZkUtils.scala | 26 +++++++++------------- 6 files changed, 69 insertions(+), 34 deletions(-) create mode 100644 core/src/main/scala/kafka/network/ConnectionType.scala diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index a0827f5..e535398 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.KafkaException; public class Utils { - private static final Pattern HOST_PORT_PATTERN = Pattern.compile("\\[?(.+?)\\]?:(\\d+)"); + private static final Pattern HOST_PORT_PATTERN = Pattern.compile("\\[?(.+?)\\]?:(\\d+):(.+?)"); public static String NL = System.getProperty("line.separator"); @@ -242,15 +242,25 @@ public class Utils { } /** + * Extracts the hostname from a "host:port" address string. + * @param address address string to parse + * @return hostname or null if the given address is incorrect + */ + public static String getConnectionType(String address) { + Matcher matcher = HOST_PORT_PATTERN.matcher(address); + return matcher.matches() ? matcher.group(3) : null; + } + + /** * Formats hostname and port number as a "host:port" address string, * surrounding IPv6 addresses with braces '[', ']' * @param host hostname * @param port port number * @return address string */ - public static String formatAddress(String host, Integer port) { + public static String formatAddress(String host, Integer port, String connectionType) { return host.contains(":") - ? "[" + host + "]:" + port // IPv6 - : host + ":" + port; + ? "[" + host + "]:" + port + connectionType // IPv6 + : host + ":" + port + ":" + connectionType; } } diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index ebba87f..f0612d0 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -24,11 +24,11 @@ import kafka.common.{ErrorMapping, KafkaException} import kafka.utils.{Utils, Logging} import java.util.Properties import util.Random - import kafka.network.BlockingChannel + import kafka.network.{ConnectionType, BlockingChannel} import kafka.utils.ZkUtils._ import org.I0Itec.zkclient.ZkClient import java.io.IOException -import org.apache.kafka.common.utils.Utils.{getHost, getPort} +import org.apache.kafka.common.utils.Utils.{getHost, getPort, getConnectionType} /** * Helper functions common to clients (producer, consumer, or admin) @@ -94,13 +94,13 @@ object ClientUtils extends Logging{ } /** - * Parse a list of broker urls in the form host1:port1, host2:port2, ... + * Parse a list of broker urls in the form host1:port1:type, host2:port2:type, ... */ def parseBrokerList(brokerListStr: String): Seq[Broker] = { val brokersStr = Utils.parseCsvList(brokerListStr) brokersStr.zipWithIndex.map { case (address, brokerId) => - new Broker(brokerId, getHost(address), getPort(address)) + new Broker(brokerId, getHost(address), getPort(address), ConnectionType.getConnectionType(getConnectionType(address))) } } diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 0060add..71ec27d 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -17,6 +17,7 @@ package kafka.cluster +import kafka.network.ConnectionType import kafka.utils.Utils._ import kafka.utils.Json import kafka.api.ApiUtils._ @@ -38,7 +39,8 @@ object Broker { val brokerInfo = m.asInstanceOf[Map[String, Any]] val host = brokerInfo.get("host").get.asInstanceOf[String] val port = brokerInfo.get("port").get.asInstanceOf[Int] - new Broker(id, host, port) + val connectionType = brokerInfo.get("type").get.asInstanceOf[String] + new Broker(id, host, port, ConnectionType.getConnectionType(connectionType)) case None => throw new BrokerNotAvailableException("Broker id %d does not exist".format(id)) } @@ -51,32 +53,34 @@ object Broker { val id = buffer.getInt val host = readShortString(buffer) val port = buffer.getInt - new Broker(id, host, port) + val connectionType = readShortString(buffer) + new Broker(id, host, port, ConnectionType.getConnectionType(connectionType)) } } -case class Broker(id: Int, host: String, port: Int) { +case class Broker(id: Int, host: String, port: Int, connectionType: ConnectionType) { - override def toString: String = "id:" + id + ",host:" + host + ",port:" + port + override def toString: String = "id:" + id + ",host:" + host + ",port:" + port + ",type:" + connectionType.name - def connectionString: String = formatAddress(host, port) + def connectionString: String = formatAddress(host, port, connectionType.name) def writeTo(buffer: ByteBuffer) { buffer.putInt(id) writeShortString(buffer, host) buffer.putInt(port) + writeShortString(buffer, connectionType.name) } - def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ + def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ + shortStringLength(connectionType.name) /* connection type */ override def equals(obj: Any): Boolean = { obj match { case null => false - case n: Broker => id == n.id && host == n.host && port == n.port + case n: Broker => id == n.id && host == n.host && port == n.port && connectionType == n.connectionType case _ => false } } - override def hashCode(): Int = hashcode(id, host, port) + override def hashCode(): Int = hashcode(id, host, port, connectionType) } diff --git a/core/src/main/scala/kafka/network/ConnectionType.scala b/core/src/main/scala/kafka/network/ConnectionType.scala new file mode 100644 index 0000000..c4baa4d --- /dev/null +++ b/core/src/main/scala/kafka/network/ConnectionType.scala @@ -0,0 +1,23 @@ +package kafka.network + +object ConnectionType { + def getConnectionType(name: String): ConnectionType = { + name.toLowerCase match { + case PlaintextConnectionType.name => PlaintextConnectionType + case SSLConnectionType.name => SSLConnectionType + case _ => throw new RuntimeException("%s is an unknown connection type".format(name)) + } + } + + def available() = Set(PlaintextConnectionType, SSLConnectionType) +} + +sealed trait ConnectionType { def name: String } + +case object PlaintextConnectionType extends ConnectionType { + val name = "plaintext" +} + +case object SSLConnectionType extends ConnectionType { + val name = "ssl" +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 4acdd70..0d67548 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -17,6 +17,7 @@ package kafka.server +import kafka.network.ConnectionType import kafka.utils._ import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkStateListener, ZkClient} @@ -26,14 +27,14 @@ import java.net.InetAddress /** * This class registers the broker in zookeeper to allow * other brokers and consumers to detect failures. It uses an ephemeral znode with the path: - * /brokers/[0...N] --> advertisedHost:advertisedPort + * /brokers/[0...N] --> advertisedHost:advertisedPorts * * Right now our definition of health is fairly naive. If we register in zk we are healthy, otherwise * we are dead. */ class KafkaHealthcheck(private val brokerId: Int, private val advertisedHost: String, - private val advertisedPort: Int, + private val advertisedPorts: Map[Int, ConnectionType], private val zkSessionTimeoutMs: Int, private val zkClient: ZkClient) extends Logging { @@ -60,7 +61,8 @@ class KafkaHealthcheck(private val brokerId: Int, else advertisedHost val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt - ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedHostName, advertisedPort, zkSessionTimeoutMs, jmxPort) + advertisedPorts.foreach(entry => ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedHostName, + entry._1, entry._2, zkSessionTimeoutMs, jmxPort)) } /** diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index a7b1fdc..282eea5 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -17,23 +17,19 @@ package kafka.utils +import kafka.admin._ +import kafka.api.LeaderAndIsr import kafka.cluster.{Broker, Cluster} +import kafka.common.{KafkaException, NoEpochForPartitionException, TopicAndPartition} import kafka.consumer.{ConsumerThreadId, TopicCount} +import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, ReassignedPartitionsContext} +import kafka.network.ConnectionType import org.I0Itec.zkclient.ZkClient -import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException, - ZkMarshallingError, ZkBadVersionException} +import org.I0Itec.zkclient.exception.{ZkBadVersionException, ZkMarshallingError, ZkNoNodeException, ZkNodeExistsException} import org.I0Itec.zkclient.serialize.ZkSerializer -import collection._ -import kafka.api.LeaderAndIsr import org.apache.zookeeper.data.Stat -import kafka.admin._ -import kafka.common.{KafkaException, NoEpochForPartitionException} -import kafka.controller.ReassignedPartitionsContext -import kafka.controller.KafkaController -import scala.Some -import kafka.controller.LeaderIsrAndControllerEpoch -import kafka.common.TopicAndPartition -import scala.collection + +import scala.collection._ object ZkUtils extends Logging { val ConsumersPath = "/consumers" @@ -158,11 +154,11 @@ object ZkUtils extends Logging { } } - def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, timeout: Int, jmxPort: Int) { + def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, connectionType: ConnectionType, timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) - val expectedBroker = new Broker(id, host, port) + val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp, "type" -> connectionType.name)) + val expectedBroker = new Broker(id, host, port, connectionType) try { createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, -- 1.8.4.msysgit.0 From 775c840d9d3eb45f11a1fc0737bade4a1ee3fa8c Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Mon, 20 Oct 2014 18:30:56 +0300 Subject: [PATCH 04/12] working --- .../java/org/apache/kafka/common/utils/Utils.java | 18 ++---- core/src/main/scala/kafka/client/ClientUtils.scala | 67 ++++++++++++---------- core/src/main/scala/kafka/cluster/Broker.scala | 29 +++++----- .../BrokerChannelNotAvailableException.scala | 22 +++++++ .../main/scala/kafka/consumer/SimpleConsumer.scala | 2 +- .../main/scala/kafka/network/BlockingChannel.scala | 43 +++++++------- .../main/scala/kafka/network/ChannelFactory.scala | 4 +- .../src/main/scala/kafka/network/ChannelInfo.scala | 57 ++++++++++++++++++ .../src/main/scala/kafka/network/ChannelType.scala | 24 ++++++++ .../main/scala/kafka/network/ConnectionType.scala | 23 -------- .../kafka/network/PlainSocketChannelFactory.scala | 2 +- .../kafka/network/ssl/SSLChannelFactory.scala | 2 +- .../main/scala/kafka/server/KafkaHealthcheck.scala | 21 ++++--- core/src/main/scala/kafka/utils/ZkUtils.scala | 35 +++++++++-- 14 files changed, 230 insertions(+), 119 deletions(-) create mode 100644 core/src/main/scala/kafka/common/BrokerChannelNotAvailableException.scala create mode 100644 core/src/main/scala/kafka/network/ChannelInfo.scala create mode 100644 core/src/main/scala/kafka/network/ChannelType.scala delete mode 100644 core/src/main/scala/kafka/network/ConnectionType.scala diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index e535398..a0827f5 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.KafkaException; public class Utils { - private static final Pattern HOST_PORT_PATTERN = Pattern.compile("\\[?(.+?)\\]?:(\\d+):(.+?)"); + private static final Pattern HOST_PORT_PATTERN = Pattern.compile("\\[?(.+?)\\]?:(\\d+)"); public static String NL = System.getProperty("line.separator"); @@ -242,25 +242,15 @@ public class Utils { } /** - * Extracts the hostname from a "host:port" address string. - * @param address address string to parse - * @return hostname or null if the given address is incorrect - */ - public static String getConnectionType(String address) { - Matcher matcher = HOST_PORT_PATTERN.matcher(address); - return matcher.matches() ? matcher.group(3) : null; - } - - /** * Formats hostname and port number as a "host:port" address string, * surrounding IPv6 addresses with braces '[', ']' * @param host hostname * @param port port number * @return address string */ - public static String formatAddress(String host, Integer port, String connectionType) { + public static String formatAddress(String host, Integer port) { return host.contains(":") - ? "[" + host + "]:" + port + connectionType // IPv6 - : host + ":" + port + ":" + connectionType; + ? "[" + host + "]:" + port // IPv6 + : host + ":" + port; } } diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index f0612d0..3ea1d01 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -16,19 +16,21 @@ */ package kafka.client -import scala.collection._ -import kafka.cluster._ +import java.io.IOException +import java.util.Properties + import kafka.api._ -import kafka.producer._ +import kafka.cluster._ import kafka.common.{ErrorMapping, KafkaException} -import kafka.utils.{Utils, Logging} -import java.util.Properties -import util.Random - import kafka.network.{ConnectionType, BlockingChannel} - import kafka.utils.ZkUtils._ - import org.I0Itec.zkclient.ZkClient - import java.io.IOException -import org.apache.kafka.common.utils.Utils.{getHost, getPort, getConnectionType} +import kafka.network.{BlockingChannel, ChannelType, PlaintextChannelType} +import kafka.producer._ +import kafka.utils.ZkUtils._ +import kafka.utils.{Logging, Utils} +import org.I0Itec.zkclient.ZkClient +import org.apache.kafka.common.utils.Utils.{getHost, getPort} + +import scala.collection._ +import scala.util.Random /** * Helper functions common to clients (producer, consumer, or admin) @@ -100,31 +102,38 @@ object ClientUtils extends Logging{ val brokersStr = Utils.parseCsvList(brokerListStr) brokersStr.zipWithIndex.map { case (address, brokerId) => - new Broker(brokerId, getHost(address), getPort(address), ConnectionType.getConnectionType(getConnectionType(address))) + new Broker(brokerId, getHost(address), getPort(address)) } } /** * Creates a blocking channel to a random broker */ - def channelToAnyBroker(zkClient: ZkClient, socketTimeoutMs: Int = 3000) : BlockingChannel = { + def channelToAnyBroker(zkClient: ZkClient, channelType: ChannelType = PlaintextChannelType, socketTimeoutMs: Int = 3000) : BlockingChannel = { var channel: BlockingChannel = null var connected = false while (!connected) { val allBrokers = getAllBrokersInCluster(zkClient) Random.shuffle(allBrokers).find { broker => - trace("Connecting to broker %s:%d.".format(broker.host, broker.port)) - try { - channel = new BlockingChannel(broker.host, broker.port, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, socketTimeoutMs) - channel.connect() - debug("Created channel to broker %s:%d.".format(channel.host, channel.port)) - true - } catch { - case e: Exception => - if (channel != null) channel.disconnect() - channel = null - info("Error while creating channel to %s:%d.".format(broker.host, broker.port)) - false + val channels = getBrokerChannels(zkClient, broker.id).filter(_.channelType == channelType) + if (channels.nonEmpty) { + val channelInfo = channels.head + trace("Connecting to broker %s:%d.".format(broker.host, channelInfo.port)) + try { + channel = new BlockingChannel(broker.host, channelInfo.port, channelInfo.channelType, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, socketTimeoutMs) + channel.connect() + debug("Created %s channel to broker %s:%d.".format(channelInfo.channelType.name, channel.host, channelInfo.port)) + true + } catch { + case e: Exception => + if (channel != null) channel.disconnect() + channel = null + info("Error while creating %s channel to %s:%d.".format(channelInfo.channelType.name, broker.host, channelInfo.port)) + false + } + } else { + //TODO: maybe we should throw an exception here? + false } } connected = if (channel == null) false else true @@ -136,8 +145,8 @@ object ClientUtils extends Logging{ /** * Creates a blocking channel to the offset manager of the given group */ - def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = { - var queryChannel = channelToAnyBroker(zkClient) + def channelToOffsetManager(group: String, channelType: ChannelType = PlaintextChannelType, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = { + var queryChannel = channelToAnyBroker(zkClient, channelType) var offsetManagerChannelOpt: Option[BlockingChannel] = None @@ -148,7 +157,7 @@ object ClientUtils extends Logging{ while (!coordinatorOpt.isDefined) { try { if (!queryChannel.isConnected) - queryChannel = channelToAnyBroker(zkClient) + queryChannel = channelToAnyBroker(zkClient, channelType) debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group)) queryChannel.send(ConsumerMetadataRequest(group)) val response = queryChannel.receive() @@ -177,7 +186,7 @@ object ClientUtils extends Logging{ var offsetManagerChannel: BlockingChannel = null try { debug("Connecting to offset manager %s.".format(connectString)) - offsetManagerChannel = new BlockingChannel(coordinator.host, coordinator.port, + offsetManagerChannel = new BlockingChannel(coordinator.host, coordinator.port, channelType, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, socketTimeoutMs) diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 71ec27d..d0ec482 100644 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -17,12 +17,12 @@ package kafka.cluster -import kafka.network.ConnectionType -import kafka.utils.Utils._ -import kafka.utils.Json -import kafka.api.ApiUtils._ import java.nio.ByteBuffer -import kafka.common.{KafkaException, BrokerNotAvailableException} + +import kafka.api.ApiUtils._ +import kafka.common.{BrokerNotAvailableException, KafkaException} +import kafka.utils.Json +import kafka.utils.Utils._ import org.apache.kafka.common.utils.Utils._ /** @@ -39,8 +39,7 @@ object Broker { val brokerInfo = m.asInstanceOf[Map[String, Any]] val host = brokerInfo.get("host").get.asInstanceOf[String] val port = brokerInfo.get("port").get.asInstanceOf[Int] - val connectionType = brokerInfo.get("type").get.asInstanceOf[String] - new Broker(id, host, port, ConnectionType.getConnectionType(connectionType)) + new Broker(id, host, port) case None => throw new BrokerNotAvailableException("Broker id %d does not exist".format(id)) } @@ -53,34 +52,32 @@ object Broker { val id = buffer.getInt val host = readShortString(buffer) val port = buffer.getInt - val connectionType = readShortString(buffer) - new Broker(id, host, port, ConnectionType.getConnectionType(connectionType)) + new Broker(id, host, port) } } -case class Broker(id: Int, host: String, port: Int, connectionType: ConnectionType) { +case class Broker(id: Int, host: String, port: Int) { - override def toString: String = "id:" + id + ",host:" + host + ",port:" + port + ",type:" + connectionType.name + override def toString: String = "id:" + id + ",host:" + host + ",port:" + port - def connectionString: String = formatAddress(host, port, connectionType.name) + def connectionString: String = formatAddress(host, port) def writeTo(buffer: ByteBuffer) { buffer.putInt(id) writeShortString(buffer, host) buffer.putInt(port) - writeShortString(buffer, connectionType.name) } - def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ + shortStringLength(connectionType.name) /* connection type */ + def sizeInBytes: Int = shortStringLength(host) /* host name */ + 4 /* port */ + 4 /* broker id*/ override def equals(obj: Any): Boolean = { obj match { case null => false - case n: Broker => id == n.id && host == n.host && port == n.port && connectionType == n.connectionType + case n: Broker => id == n.id && host == n.host && port == n.port case _ => false } } - override def hashCode(): Int = hashcode(id, host, port, connectionType) + override def hashCode(): Int = hashcode(id, host, port) } diff --git a/core/src/main/scala/kafka/common/BrokerChannelNotAvailableException.scala b/core/src/main/scala/kafka/common/BrokerChannelNotAvailableException.scala new file mode 100644 index 0000000..850d266 --- /dev/null +++ b/core/src/main/scala/kafka/common/BrokerChannelNotAvailableException.scala @@ -0,0 +1,22 @@ +/** + * 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 kafka.common + +class BrokerChannelNotAvailableException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index d349a30..4f96767 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -35,7 +35,7 @@ class SimpleConsumer(val host: String, ConsumerConfig.validateClientId(clientId) private val lock = new Object() - private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout) + private val blockingChannel = new BlockingChannel(host, port, PlaintextChannelType, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout) val brokerInfo = "host_%s-port_%s".format(host, port) private val fetchRequestAndResponseStats = FetchRequestAndResponseStatsRegistry.getFetchRequestAndResponseStats(clientId) private var isClosed = false diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala index eb7bb14..6957503 100644 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ b/core/src/main/scala/kafka/network/BlockingChannel.scala @@ -6,7 +6,7 @@ * (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 + * 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, @@ -23,33 +23,34 @@ import kafka.utils.{nonthreadsafe, Logging} import kafka.api.RequestOrResponse -object BlockingChannel{ +object BlockingChannel { val UseDefaultBufferSize = -1 } /** - * A simple blocking channel with timeouts correctly enabled. + * A simple blocking channel with timeouts correctly enabled. * */ @nonthreadsafe -class BlockingChannel( val host: String, - val port: Int, - val readBufferSize: Int, - val writeBufferSize: Int, - val readTimeoutMs: Int ) extends Logging { +class BlockingChannel(val host: String, + val port: Int, + val channelType: ChannelType, + val readBufferSize: Int, + val writeBufferSize: Int, + val readTimeoutMs: Int) extends Logging { private var connected = false private var channel: SocketChannel = null private var readChannel: ReadableByteChannel = null private var writeChannel: GatheringByteChannel = null private val lock = new Object() - - def connect() = lock synchronized { - if(!connected) { + + def connect() = lock synchronized { + if (!connected) { try { channel = SocketChannel.open() - if(readBufferSize > 0) + if (readBufferSize > 0) channel.socket.setReceiveBufferSize(readBufferSize) - if(writeBufferSize > 0) + if (writeBufferSize > 0) channel.socket.setSendBufferSize(writeBufferSize) channel.configureBlocking(true) channel.socket.setSoTimeout(readTimeoutMs) @@ -64,7 +65,7 @@ class BlockingChannel( val host: String, val msg = "Created socket with SO_TIMEOUT = %d (requested %d), SO_RCVBUF = %d (requested %d), SO_SNDBUF = %d (requested %d)." debug(msg.format(channel.socket.getSoTimeout, readTimeoutMs, - channel.socket.getReceiveBufferSize, + channel.socket.getReceiveBufferSize, readBufferSize, channel.socket.getSendBufferSize, writeBufferSize)) @@ -73,9 +74,9 @@ class BlockingChannel( val host: String, } } } - + def disconnect() = lock synchronized { - if(channel != null) { + if (channel != null) { swallow(channel.close()) swallow(channel.socket.close()) channel = null @@ -83,7 +84,7 @@ class BlockingChannel( val host: String, } // closing the main socket channel *should* close the read channel // but let's do it to be sure. - if(readChannel != null) { + if (readChannel != null) { swallow(readChannel.close()) readChannel = null } @@ -92,16 +93,16 @@ class BlockingChannel( val host: String, def isConnected = connected - def send(request: RequestOrResponse):Int = { - if(!connected) + def send(request: RequestOrResponse): Int = { + if (!connected) throw new ClosedChannelException() val send = new BoundedByteBufferSend(request) send.writeCompletely(writeChannel) } - + def receive(): Receive = { - if(!connected) + if (!connected) throw new ClosedChannelException() val response = new BoundedByteBufferReceive() diff --git a/core/src/main/scala/kafka/network/ChannelFactory.scala b/core/src/main/scala/kafka/network/ChannelFactory.scala index 2083d93..e907cb4 100644 --- a/core/src/main/scala/kafka/network/ChannelFactory.scala +++ b/core/src/main/scala/kafka/network/ChannelFactory.scala @@ -23,12 +23,12 @@ import kafka.utils.Logging abstract class ChannelFactory extends Logging { def create(serverSocketChannel: ServerSocketChannel, recvBufferSize: Int, sendBufferSize: Int): KafkaChannel = { - val channel = createSocket(serverSocketChannel) + val channel = createChannel(serverSocketChannel) configure(channel, recvBufferSize, sendBufferSize) channel } - protected def createSocket(serverSocketChannel: ServerSocketChannel): KafkaChannel + protected def createChannel(serverSocketChannel: ServerSocketChannel): KafkaChannel protected def configure(socketChannel: SocketChannel, recvBufferSize: Int, sendBufferSize: Int) { socketChannel.configureBlocking(false) diff --git a/core/src/main/scala/kafka/network/ChannelInfo.scala b/core/src/main/scala/kafka/network/ChannelInfo.scala new file mode 100644 index 0000000..9d11ec1 --- /dev/null +++ b/core/src/main/scala/kafka/network/ChannelInfo.scala @@ -0,0 +1,57 @@ +/** + * 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 kafka.network + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.{BrokerChannelNotAvailableException, KafkaException, BrokerNotAvailableException} +import kafka.utils.Json + +object ChannelInfo { + + def createChannelInfo(brokerId: Int, channelInfoString: String): ChannelInfo = { + if(channelInfoString == null) + throw new BrokerNotAvailableException("Channel for broker id %s does not exist".format(brokerId)) + try { + Json.parseFull(channelInfoString) match { + case Some(m) => + val channelInfo = m.asInstanceOf[Map[String, Any]] + val brokerId = channelInfo.get("brokerId").get.asInstanceOf[Int] + val port = channelInfo.get("port").get.asInstanceOf[Int] + val channelType = ChannelType.getChannelType(channelInfo.get("type").get.asInstanceOf[String]) + new ChannelInfo(brokerId, port, channelType) + case None => + throw new BrokerChannelNotAvailableException("Broker id %d does not exist".format(brokerId)) + } + } catch { + case t: Throwable => throw new KafkaException("Failed to parse the broker channel info from zookeeper: " + channelInfoString, t) + } + } + + def readFrom(buffer: ByteBuffer): ChannelInfo = { + val brokerId = buffer.getInt + val port = buffer.getInt + val channelType = readShortString(buffer) + new ChannelInfo(brokerId, port, ChannelType.getChannelType(channelType)) + } +} + +class ChannelInfo(val brokerId: Int, val port: Int, val channelType: ChannelType) { + override def toString: String = "brokerId:%d,port:%d,type:%s".format(brokerId, port, channelType) +} diff --git a/core/src/main/scala/kafka/network/ChannelType.scala b/core/src/main/scala/kafka/network/ChannelType.scala new file mode 100644 index 0000000..ff64e1a --- /dev/null +++ b/core/src/main/scala/kafka/network/ChannelType.scala @@ -0,0 +1,24 @@ +package kafka.network + +object ChannelType { + def getChannelType(name: String): ChannelType = { + name.toLowerCase match { + case PlaintextChannelType.name => PlaintextChannelType + case SSLChannelType.name => SSLChannelType + case _ => throw new RuntimeException("%s is an unknown channel type".format(name)) + } + } + + def available() = Set(PlaintextChannelType, SSLChannelType) +} + +sealed trait ChannelType { def name: String } + + +case object PlaintextChannelType extends ChannelType { + val name = "plaintext" +} + +case object SSLChannelType extends ChannelType { + val name = "ssl" +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/network/ConnectionType.scala b/core/src/main/scala/kafka/network/ConnectionType.scala deleted file mode 100644 index c4baa4d..0000000 --- a/core/src/main/scala/kafka/network/ConnectionType.scala +++ /dev/null @@ -1,23 +0,0 @@ -package kafka.network - -object ConnectionType { - def getConnectionType(name: String): ConnectionType = { - name.toLowerCase match { - case PlaintextConnectionType.name => PlaintextConnectionType - case SSLConnectionType.name => SSLConnectionType - case _ => throw new RuntimeException("%s is an unknown connection type".format(name)) - } - } - - def available() = Set(PlaintextConnectionType, SSLConnectionType) -} - -sealed trait ConnectionType { def name: String } - -case object PlaintextConnectionType extends ConnectionType { - val name = "plaintext" -} - -case object SSLConnectionType extends ConnectionType { - val name = "ssl" -} \ No newline at end of file diff --git a/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala b/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala index fb605ea..968a78a 100644 --- a/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala +++ b/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala @@ -21,7 +21,7 @@ import java.nio.channels.ServerSocketChannel class PlainSocketChannelFactory extends ChannelFactory { - protected def createSocket(serverSocketChannel: ServerSocketChannel): KafkaChannel = { + protected def createChannel(serverSocketChannel: ServerSocketChannel): KafkaChannel = { new KafkaChannel(serverSocketChannel.accept()) } } diff --git a/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala b/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala index 6261607..1d980fc 100644 --- a/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala +++ b/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala @@ -23,7 +23,7 @@ import kafka.network.{ChannelFactory, KafkaChannel} class SSLChannelFactory(wantClientAuth: Boolean, needClientAuth: Boolean) extends ChannelFactory { - protected def createSocket(serverSocketChannel: ServerSocketChannel): KafkaChannel = { + protected def createChannel(serverSocketChannel: ServerSocketChannel): KafkaChannel = { SSLSocketChannel.makeSecureServerConnection(serverSocketChannel.accept(), wantClientAuth, needClientAuth) diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 0d67548..d07123c 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -17,24 +17,28 @@ package kafka.server -import kafka.network.ConnectionType +import java.net.InetAddress + +import kafka.network.{ChannelInfo, PlaintextChannelType} import kafka.utils._ -import org.apache.zookeeper.Watcher.Event.KeeperState import org.I0Itec.zkclient.{IZkStateListener, ZkClient} -import java.net.InetAddress +import org.apache.zookeeper.Watcher.Event.KeeperState /** * This class registers the broker in zookeeper to allow * other brokers and consumers to detect failures. It uses an ephemeral znode with the path: - * /brokers/[0...N] --> advertisedHost:advertisedPorts + * /brokers/[0...N] --> advertisedHost:advertisedPort * * Right now our definition of health is fairly naive. If we register in zk we are healthy, otherwise * we are dead. */ class KafkaHealthcheck(private val brokerId: Int, private val advertisedHost: String, - private val advertisedPorts: Map[Int, ConnectionType], + private val advertisedPort: Int, + private val channels: List[ChannelInfo] = List(new ChannelInfo(brokerId, + advertisedPort, + PlaintextChannelType)), private val zkSessionTimeoutMs: Int, private val zkClient: ZkClient) extends Logging { @@ -61,8 +65,11 @@ class KafkaHealthcheck(private val brokerId: Int, else advertisedHost val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt - advertisedPorts.foreach(entry => ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedHostName, - entry._1, entry._2, zkSessionTimeoutMs, jmxPort)) + ZkUtils.registerBrokerInZk(zkClient, brokerId, advertisedHostName, advertisedPort, zkSessionTimeoutMs, jmxPort) + channels.foreach(channelInfo => + ZkUtils.registerBrokerChannelInZk(zkClient, + channelInfo.brokerId, channelInfo.port, channelInfo.channelType, + zkSessionTimeoutMs)) } /** diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 282eea5..d17968a 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -23,7 +23,7 @@ import kafka.cluster.{Broker, Cluster} import kafka.common.{KafkaException, NoEpochForPartitionException, TopicAndPartition} import kafka.consumer.{ConsumerThreadId, TopicCount} import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, ReassignedPartitionsContext} -import kafka.network.ConnectionType +import kafka.network.{ChannelInfo, ChannelType} import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.{ZkBadVersionException, ZkMarshallingError, ZkNoNodeException, ZkNodeExistsException} import org.I0Itec.zkclient.serialize.ZkSerializer @@ -34,6 +34,7 @@ import scala.collection._ object ZkUtils extends Logging { val ConsumersPath = "/consumers" val BrokerIdsPath = "/brokers/ids" + val BrokerChannelsPath = "/brokers/channels" val BrokerTopicsPath = "/brokers/topics" val TopicConfigPath = "/config/topics" val TopicConfigChangesPath = "/config/changes" @@ -154,11 +155,11 @@ object ZkUtils extends Logging { } } - def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, connectionType: ConnectionType, timeout: Int, jmxPort: Int) { + def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, port: Int, timeout: Int, jmxPort: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp, "type" -> connectionType.name)) - val expectedBroker = new Broker(id, host, port, connectionType) + val brokerInfo = Json.encode(Map("version" -> 1, "host" -> host, "port" -> port, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val expectedBroker = new Broker(id, host, port) try { createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerIdPath, brokerInfo, expectedBroker, @@ -175,6 +176,24 @@ object ZkUtils extends Logging { info("Registered broker %d at path %s with address %s:%d.".format(id, brokerIdPath, host, port)) } + def registerBrokerChannelInZk(zkClient: ZkClient, brokerId: Int, port: Int, channelType: ChannelType, timeout: Int) { + val brokerChannelsIdPath = ZkUtils.BrokerChannelsPath + "/" + brokerId + "/" + channelType.name + val channelInfo = Json.encode(Map("version" -> 1, "brokerId" -> brokerId, "port" -> port, "type" -> channelType.name)) + val expectedChannel = new ChannelInfo(brokerId, port, channelType) + + try { + createEphemeralPathExpectConflictHandleZKBug(zkClient, brokerChannelsIdPath, channelInfo, expectedChannel, + (channelString: String, channel: Any) => ChannelInfo.createChannelInfo(channel.asInstanceOf[ChannelInfo].brokerId, channelString).equals(channel.asInstanceOf[ChannelInfo]), + timeout) + + } catch { + case e: ZkNodeExistsException => + //TODO: think of when could it happen + throw new RuntimeException("") + } + info("Registered channel of type %s for broker %d at path %s on port %d.".format(channelType.name, brokerId, brokerChannelsIdPath, port)) + } + def deregisterBrokerInZk(zkClient: ZkClient, id: Int) { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id deletePath(zkClient, brokerIdPath) @@ -687,6 +706,14 @@ object ZkUtils extends Logging { } } + def getBrokerChannels(zkClient: ZkClient, brokerId: Int): Seq[ChannelInfo] = { + val channels = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerChannelsPath + "/" + brokerId) + if(channels == null) + Seq.empty[ChannelInfo] + else + channels.map { channel => ChannelInfo.createChannelInfo(brokerId, channel) } + } + def getAllTopics(zkClient: ZkClient): Seq[String] = { val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) if(topics == null) -- 1.8.4.msysgit.0 From 28b8f5c375e8918ed1ce75a39d5e71a6c8a75d07 Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Mon, 20 Oct 2014 21:07:27 +0300 Subject: [PATCH 05/12] working --- config/log4j.properties | 10 +++--- .../main/scala/kafka/consumer/ConsumerConfig.scala | 4 +++ .../consumer/ZookeeperConsumerConnector.scala | 2 +- .../controller/ControllerChannelManager.scala | 30 ++++++++-------- .../main/scala/kafka/network/BlockingChannel.scala | 10 +----- .../main/scala/kafka/network/ChannelFactory.scala | 29 +++++++++++++--- .../src/main/scala/kafka/network/ChannelType.scala | 6 +++- .../kafka/network/PlainSocketChannelFactory.scala | 10 ++++-- .../main/scala/kafka/network/SocketServer.scala | 12 +++---- .../kafka/network/ssl/SSLChannelFactory.scala | 30 ++++++++++++---- .../main/scala/kafka/producer/ProducerConfig.scala | 12 ++++--- .../main/scala/kafka/producer/SyncProducer.scala | 2 +- .../scala/kafka/producer/SyncProducerConfig.scala | 4 +++ core/src/main/scala/kafka/server/KafkaConfig.scala | 20 ++++++++++- .../main/scala/kafka/server/KafkaHealthcheck.scala | 6 ++-- core/src/main/scala/kafka/server/KafkaServer.scala | 40 ++++++++-------------- .../main/scala/kafka/tools/ConsoleProducer.scala | 8 +++++ .../scala/kafka/tools/ConsumerOffsetChecker.scala | 22 ++++++------ 18 files changed, 161 insertions(+), 96 deletions(-) diff --git a/config/log4j.properties b/config/log4j.properties index 9502254..9987359 100644 --- a/config/log4j.properties +++ b/config/log4j.properties @@ -52,11 +52,11 @@ log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n # Turn on all our debugging info -#log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender -#log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender -#log4j.logger.kafka.perf=DEBUG, kafkaAppender -#log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG, kafkaAppender -#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG +log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender +log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender +log4j.logger.kafka.perf=DEBUG, kafkaAppender +log4j.logger.kafka.perf.ProducerPerformance$ProducerThread=DEBUG, kafkaAppender +log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG log4j.logger.kafka=INFO, kafkaAppender log4j.logger.kafka.network.RequestChannel$=WARN, requestAppender diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 9ebbee6..2c0ab8e 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -19,6 +19,7 @@ package kafka.consumer import java.util.Properties import kafka.api.OffsetRequest +import kafka.network.ChannelType import kafka.utils._ import kafka.common.{InvalidConfigException, Config} @@ -180,6 +181,9 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( /** Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin */ val partitionAssignmentStrategy = props.getString("partition.assignment.strategy", DefaultPartitionAssignmentStrategy) + + /* plaintext or SSL */ + val channelType = ChannelType.getChannelType(props.getString("channel", "plaintext")) validate(this) } diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala index fbc680f..fc64f01 100644 --- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala +++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala @@ -174,7 +174,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig, private def ensureOffsetManagerConnected() { if (config.offsetsStorage == "kafka") { if (offsetsChannel == null || !offsetsChannel.isConnected) - offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient, config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs) + offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, config.channelType, zkClient, config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs) debug("Connected to offset manager %s:%d.".format(offsetsChannel.host, offsetsChannel.port)) } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index ecbfa0f..5a736db 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -16,19 +16,17 @@ */ package kafka.controller -import kafka.network.{Receive, BlockingChannel} -import kafka.utils.{Utils, Logging, ShutdownableThread} -import collection.mutable.HashMap +import java.util.concurrent.{BlockingQueue, LinkedBlockingQueue} + +import kafka.api.{RequestOrResponse, _} import kafka.cluster.Broker -import java.util.concurrent.{LinkedBlockingQueue, BlockingQueue} -import kafka.server.KafkaConfig -import collection.mutable -import kafka.api._ -import org.apache.log4j.Logger -import scala.Some import kafka.common.TopicAndPartition -import kafka.api.RequestOrResponse -import collection.Set +import kafka.network.{BlockingChannel, Receive} +import kafka.server.KafkaConfig +import kafka.utils.{Logging, ShutdownableThread, Utils} + +import scala.collection.{Set, mutable} +import scala.collection.mutable.HashMap class ControllerChannelManager (private val controllerContext: ControllerContext, config: KafkaConfig) extends Logging { private val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] @@ -80,10 +78,12 @@ class ControllerChannelManager (private val controllerContext: ControllerContext private def addNewBroker(broker: Broker) { val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize) debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id)) - val channel = new BlockingChannel(broker.host, broker.port, - BlockingChannel.UseDefaultBufferSize, - BlockingChannel.UseDefaultBufferSize, - config.controllerSocketTimeoutMs) + val channel = new BlockingChannel(broker.host, + config.portsToChannelTypes.head._1, + config.portsToChannelTypes.head._2, + BlockingChannel.UseDefaultBufferSize, + BlockingChannel.UseDefaultBufferSize, + config.controllerSocketTimeoutMs) val requestThread = new RequestSendThread(config.brokerId, controllerContext, broker, messageQueue, channel) requestThread.setDaemon(false) brokerStateInfo.put(broker.id, new ControllerBrokerStateInfo(channel, broker, messageQueue, requestThread)) diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala index 6957503..1de922c 100644 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ b/core/src/main/scala/kafka/network/BlockingChannel.scala @@ -47,15 +47,7 @@ class BlockingChannel(val host: String, def connect() = lock synchronized { if (!connected) { try { - channel = SocketChannel.open() - if (readBufferSize > 0) - channel.socket.setReceiveBufferSize(readBufferSize) - if (writeBufferSize > 0) - channel.socket.setSendBufferSize(writeBufferSize) - channel.configureBlocking(true) - channel.socket.setSoTimeout(readTimeoutMs) - channel.socket.setKeepAlive(true) - channel.socket.setTcpNoDelay(true) + channel = channelType.factory.createClientChannel(host, port, readBufferSize, writeBufferSize, readTimeoutMs) channel.connect(new InetSocketAddress(host, port)) writeChannel = channel diff --git a/core/src/main/scala/kafka/network/ChannelFactory.scala b/core/src/main/scala/kafka/network/ChannelFactory.scala index e907cb4..601468f 100644 --- a/core/src/main/scala/kafka/network/ChannelFactory.scala +++ b/core/src/main/scala/kafka/network/ChannelFactory.scala @@ -22,15 +22,23 @@ import java.nio.channels.{SocketChannel, ServerSocketChannel} import kafka.utils.Logging abstract class ChannelFactory extends Logging { - def create(serverSocketChannel: ServerSocketChannel, recvBufferSize: Int, sendBufferSize: Int): KafkaChannel = { - val channel = createChannel(serverSocketChannel) - configure(channel, recvBufferSize, sendBufferSize) + def createServerChannel(serverSocketChannel: ServerSocketChannel, recvBufferSize: Int, sendBufferSize: Int): KafkaChannel = { + val channel = createServerChannelImpl(serverSocketChannel) + configureServerChannel(channel, recvBufferSize, sendBufferSize) channel } + + def createClientChannel(host: String, port: Int, readBufferSize: Int, writeBufferSize: Int, readTimeoutMs: Int): SocketChannel = { + val channel = createClientChannelImpl(host, port) + configureClientChannel(channel, readBufferSize, writeBufferSize, readTimeoutMs) + channel + } + + protected def createServerChannelImpl(serverSocketChannel: ServerSocketChannel): KafkaChannel - protected def createChannel(serverSocketChannel: ServerSocketChannel): KafkaChannel + protected def createClientChannelImpl(host: String, port: Int): SocketChannel - protected def configure(socketChannel: SocketChannel, recvBufferSize: Int, sendBufferSize: Int) { + protected def configureServerChannel(socketChannel: SocketChannel, recvBufferSize: Int, sendBufferSize: Int) { socketChannel.configureBlocking(false) socketChannel.socket().setTcpNoDelay(true) socketChannel.socket().setSendBufferSize(sendBufferSize) @@ -40,4 +48,15 @@ abstract class ChannelFactory extends Logging { socketChannel.socket.getSendBufferSize, sendBufferSize, socketChannel.socket.getReceiveBufferSize, recvBufferSize)) } + + protected def configureClientChannel(channel: SocketChannel, readBufferSize: Int, writeBufferSize: Int, readTimeoutMs: Int) = { + if (readBufferSize > 0) + channel.socket.setReceiveBufferSize(readBufferSize) + if (writeBufferSize > 0) + channel.socket.setSendBufferSize(writeBufferSize) + channel.configureBlocking(true) + channel.socket.setSoTimeout(readTimeoutMs) + channel.socket.setKeepAlive(true) + channel.socket.setTcpNoDelay(true) + } } diff --git a/core/src/main/scala/kafka/network/ChannelType.scala b/core/src/main/scala/kafka/network/ChannelType.scala index ff64e1a..098c56c 100644 --- a/core/src/main/scala/kafka/network/ChannelType.scala +++ b/core/src/main/scala/kafka/network/ChannelType.scala @@ -1,5 +1,7 @@ package kafka.network +import kafka.network.ssl.SSLChannelFactory + object ChannelType { def getChannelType(name: String): ChannelType = { name.toLowerCase match { @@ -12,13 +14,15 @@ object ChannelType { def available() = Set(PlaintextChannelType, SSLChannelType) } -sealed trait ChannelType { def name: String } +sealed trait ChannelType { def name: String; def factory: ChannelFactory } case object PlaintextChannelType extends ChannelType { val name = "plaintext" + override val factory = PlainSocketChannelFactory } case object SSLChannelType extends ChannelType { val name = "ssl" + override val factory = SSLChannelFactory } \ No newline at end of file diff --git a/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala b/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala index 968a78a..44c8773 100644 --- a/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala +++ b/core/src/main/scala/kafka/network/PlainSocketChannelFactory.scala @@ -17,11 +17,15 @@ package kafka.network -import java.nio.channels.ServerSocketChannel +import java.nio.channels.{SocketChannel, ServerSocketChannel} -class PlainSocketChannelFactory extends ChannelFactory { +object PlainSocketChannelFactory extends ChannelFactory { - protected def createChannel(serverSocketChannel: ServerSocketChannel): KafkaChannel = { + protected def createServerChannelImpl(serverSocketChannel: ServerSocketChannel): KafkaChannel = { new KafkaChannel(serverSocketChannel.accept()) } + + protected def createClientChannelImpl(host: String, port: Int): SocketChannel = { + SocketChannel.open() + } } diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 50dccd3..e3e34e0 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -39,7 +39,7 @@ import scala.collection._ */ class SocketServer(val brokerId: Int, val host: String, - val ports: Map[Int, ChannelFactory], + val ports: Map[Int, ChannelType], val numProcessorThreads: Int, val maxQueuedRequests: Int, val sendBufferSize: Int, @@ -72,7 +72,7 @@ class SocketServer(val brokerId: Int, maxConnectionsPerIpOverrides: Map[String, Int] = Map.empty[String, Int]) { this(brokerId, host, - Map(port -> new PlainSocketChannelFactory()), + Map(port -> PlaintextChannelType), numProcessorThreads, maxQueuedRequests, sendBufferSize, @@ -222,7 +222,7 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ * Thread that accepts and configures new connections. There is only need for one of these */ private[kafka] class Acceptor(val host: String, - val ports: Map[Int, ChannelFactory], + val ports: Map[Int, ChannelType], private val processors: Array[Processor], val sendBufferSize: Int, val recvBufferSize: Int, @@ -233,10 +233,10 @@ private[kafka] class Acceptor(val host: String, * Accept loop that checks for new connection attempts */ def run() { - for ((port, channelFactory) <- ports) { + for ((port, channelType) <- ports) { val serverChannel = openServerSocket(host, port) serverChannel.register(selector, SelectionKey.OP_ACCEPT) - channelToFactory += serverChannel -> channelFactory + channelToFactory += serverChannel -> channelType.factory } startupComplete() @@ -300,7 +300,7 @@ private[kafka] class Acceptor(val host: String, var socketChannel: Option[KafkaChannel] = None try { val channelFactory = channelToFactory(serverSocketChannel) - socketChannel = Some(channelFactory.create(serverSocketChannel, recvBufferSize, sendBufferSize)) + socketChannel = Some(channelFactory.createServerChannel(serverSocketChannel, recvBufferSize, sendBufferSize)) connectionQuotas.inc(socketChannel.get.socket().getInetAddress) processor.accept(socketChannel.get) } catch { diff --git a/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala b/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala index 1d980fc..853f4a4 100644 --- a/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala +++ b/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala @@ -17,15 +17,33 @@ package kafka.network.ssl -import java.nio.channels.ServerSocketChannel +import java.nio.channels.{SocketChannel, ServerSocketChannel} import kafka.network.{ChannelFactory, KafkaChannel} -class SSLChannelFactory(wantClientAuth: Boolean, - needClientAuth: Boolean) extends ChannelFactory { - protected def createChannel(serverSocketChannel: ServerSocketChannel): KafkaChannel = { +object SSLChannelFactory extends ChannelFactory { + private val config = SSLConnectionConfig.server + SSLAuth.initialize(config) + + protected def createServerChannelImpl(serverSocketChannel: ServerSocketChannel): KafkaChannel = { SSLSocketChannel.makeSecureServerConnection(serverSocketChannel.accept(), - wantClientAuth, - needClientAuth) + config.wantClientAuth, + config.needClientAuth) + } + + protected def createClientChannelImpl(host: String, port: Int): SocketChannel = { + SSLSocketChannel.makeSecureClientConnection(SocketChannel.open(), host, port) + } + + override protected def configureClientChannel(channel: SocketChannel, readBufferSize: Int, writeBufferSize: Int, readTimeoutMs: Int): Unit = { + val secureChannel = channel.asInstanceOf[SSLSocketChannel] + if (readBufferSize > 0) + secureChannel.socket.setReceiveBufferSize(readBufferSize) + if (writeBufferSize > 0) + secureChannel.socket.setSendBufferSize(writeBufferSize) + secureChannel.simulateBlocking(true) + secureChannel.socket.setSoTimeout(readTimeoutMs) + secureChannel.socket.setKeepAlive(true) + secureChannel.socket.setTcpNoDelay(true) } } diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index 3cdf23d..2f39420 100644 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -17,11 +17,12 @@ package kafka.producer -import async.AsyncProducerConfig import java.util.Properties + +import kafka.common.{Config, InvalidConfigException} +import kafka.message.NoCompressionCodec +import kafka.producer.async.AsyncProducerConfig import kafka.utils.{Utils, VerifiableProperties} -import kafka.message.{CompressionCodec, NoCompressionCodec} -import kafka.common.{InvalidConfigException, Config} object ProducerConfig extends Config { def validate(config: ProducerConfig) { @@ -50,7 +51,7 @@ object ProducerConfig extends Config { class ProducerConfig private (val props: VerifiableProperties) extends AsyncProducerConfig with SyncProducerConfigShared { - import ProducerConfig._ + import kafka.producer.ProducerConfig._ def this(originalProps: Properties) { this(new VerifiableProperties(originalProps)) @@ -68,6 +69,9 @@ class ProducerConfig private (val props: VerifiableProperties) /** the partitioner class for partitioning events amongst sub-topics */ val partitionerClass = props.getString("partitioner.class", "kafka.producer.DefaultPartitioner") + /* SSL or plaintext */ + val channelType = props.getString("channel", "plaintext") + /** this parameter specifies whether the messages are sent asynchronously * * or not. Valid values are - async for asynchronous send * * sync for synchronous send */ diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index 42c9503..60fa44a 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -37,7 +37,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { private val lock = new Object() @volatile private var shutdown: Boolean = false - private val blockingChannel = new BlockingChannel(config.host, config.port, BlockingChannel.UseDefaultBufferSize, + private val blockingChannel = new BlockingChannel(config.host, config.port, config.channelType, BlockingChannel.UseDefaultBufferSize, config.sendBufferBytes, config.requestTimeoutMs) val brokerInfo = "host_%s-port_%s".format(config.host, config.port) val producerRequestStats = ProducerRequestStatsRegistry.getProducerRequestStats(config.clientId) diff --git a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala index 69b2d0c..1773a79 100644 --- a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala @@ -18,6 +18,7 @@ package kafka.producer import java.util.Properties +import kafka.network.ChannelType import kafka.utils.VerifiableProperties class SyncProducerConfig private (val props: VerifiableProperties) extends SyncProducerConfigShared { @@ -31,6 +32,9 @@ class SyncProducerConfig private (val props: VerifiableProperties) extends SyncP /** the port on which the broker is running */ val port = props.getInt("port") + + /** plaintext or SSL */ + val channelType = ChannelType.getChannelType(props.getString("channel", "plaintext")) } trait SyncProducerConfigShared { diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 01667e6..f94ab53 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -21,8 +21,12 @@ import java.util.Properties import kafka.consumer.ConsumerConfig import kafka.message.{Message, MessageSet} +import kafka.network.ChannelType +import kafka.network.ssl.SSLConnectionConfig import kafka.utils.{Utils, VerifiableProperties, ZKConfig} +import scala.collection.{mutable, Map} + /** * Configuration settings for the kafka server */ @@ -323,5 +327,19 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off */ val deleteTopicEnable = props.getBoolean("delete.topic.enable", false) - + + val portsToChannelTypes: Map[Int, ChannelType] = { + val portToChannelType = mutable.Map.empty[Int, ChannelType] + if (port > 0) { + portToChannelType += port -> ChannelType.getChannelType("plaintext") + } + if (sslEnabled) { + val sslConfig = SSLConnectionConfig.server + portToChannelType += sslConfig.port -> ChannelType.getChannelType("ssl") + } + + if (portToChannelType.isEmpty) portToChannelType += 9092 -> ChannelType.getChannelType("plaintext") + + portToChannelType.toMap + } } diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index d07123c..b4fb050 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -19,7 +19,7 @@ package kafka.server import java.net.InetAddress -import kafka.network.{ChannelInfo, PlaintextChannelType} +import kafka.network.ChannelInfo import kafka.utils._ import org.I0Itec.zkclient.{IZkStateListener, ZkClient} import org.apache.zookeeper.Watcher.Event.KeeperState @@ -36,9 +36,7 @@ import org.apache.zookeeper.Watcher.Event.KeeperState class KafkaHealthcheck(private val brokerId: Int, private val advertisedHost: String, private val advertisedPort: Int, - private val channels: List[ChannelInfo] = List(new ChannelInfo(brokerId, - advertisedPort, - PlaintextChannelType)), + private val channels: List[ChannelInfo], private val zkSessionTimeoutMs: Int, private val zkClient: ZkClient) extends Logging { diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index ab3ca22..3123825 100644 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -29,12 +29,11 @@ import kafka.common.ErrorMapping import kafka.controller.{ControllerStats, KafkaController} import kafka.log.{CleanerConfig, LogConfig, LogManager} import kafka.metrics.KafkaMetricsGroup -import kafka.network.ssl.{SSLChannelFactory, SSLConnectionConfig, SSLAuth} import kafka.network._ import kafka.utils._ import org.I0Itec.zkclient.ZkClient -import scala.collection.{Map, mutable} +import scala.collection.Map /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required @@ -89,7 +88,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg socketServer = new SocketServer(config.brokerId, config.hostName, - portsToChannelFactories, + config.portsToChannelTypes, config.numNetworkThreads, config.queuedMaxRequests, config.socketSendBufferBytes, @@ -122,7 +121,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg topicConfigManager.startup() /* tell everyone we are alive */ - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, config.zkSessionTimeoutMs, zkClient) + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, config.advertisedHostName, config.advertisedPort, + config.portsToChannelTypes.map(entry => new ChannelInfo(config.brokerId, entry._1, entry._2)).toList, + config.zkSessionTimeoutMs, zkClient) kafkaHealthcheck.startup() @@ -137,22 +138,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg throw e } } - - def portsToChannelFactories: Map[Int, ChannelFactory] = { - val portToChannelFactory = mutable.Map.empty[Int, ChannelFactory] - if (config.port > 0) { - portToChannelFactory += config.port -> new PlainSocketChannelFactory() - } - if (config.sslEnabled) { - val sslConfig = SSLConnectionConfig.server - SSLAuth.initialize(sslConfig) - portToChannelFactory += sslConfig.port -> new SSLChannelFactory(sslConfig.wantClientAuth, sslConfig.needClientAuth) - } - - if (portToChannelFactory.isEmpty) portToChannelFactory += 9092 -> new PlainSocketChannelFactory() - - portToChannelFactory.toMap - } private def initZk(): ZkClient = { info("Connecting to zookeeper on " + config.zkConnect) @@ -218,11 +203,16 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (channel != null) { channel.disconnect() } - channel = new BlockingChannel(broker.host, broker.port, - BlockingChannel.UseDefaultBufferSize, - BlockingChannel.UseDefaultBufferSize, - config.controllerSocketTimeoutMs) - channel.connect() + //TODO: maybe one is enough? + for ((port, channelType) <- config.portsToChannelTypes) { + channel = new BlockingChannel(broker.host, + port, + channelType, + BlockingChannel.UseDefaultBufferSize, + BlockingChannel.UseDefaultBufferSize, + config.controllerSocketTimeoutMs) + channel.connect() + } prevController = broker } case None=> diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index 8e9ba0b..2ae34db 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -19,6 +19,7 @@ package kafka.tools import kafka.common._ import kafka.message._ +import kafka.network.ChannelType import kafka.serializer._ import kafka.utils.{ToolsUtils, CommandLineUtils} import kafka.producer.{NewShinyProducer,OldProducer,KeyedMessage} @@ -63,6 +64,7 @@ object ConsoleProducer { new NewShinyProducer(props) } else { props.put("metadata.broker.list", config.brokerList) + props.put("channel", config.channelType.name) props.put("compression.codec", config.compressionCodec) props.put("producer.type", if(config.sync) "sync" else "async") props.put("batch.num.messages", config.batchSize.toString) @@ -112,6 +114,11 @@ object ConsoleProducer { .withRequiredArg .describedAs("broker-list") .ofType(classOf[String]) + val channelTypeOpt = parser.accepts("channel", "plaintext or SSL") + .withRequiredArg() + .describedAs("channel") + .defaultsTo("plaintext") + .ofType(classOf[String]) val syncOpt = parser.accepts("sync", "If set message send requests to the brokers are synchronously, one at a time as they arrive.") val compressionCodecOpt = parser.accepts("compression-codec", "The compression codec: either 'gzip' or 'snappy'." + "If specified without value, than it defaults to 'gzip'") @@ -221,6 +228,7 @@ object ConsoleProducer { val topic = options.valueOf(topicOpt) val brokerList = options.valueOf(brokerListOpt) ToolsUtils.validatePortOrDie(parser,brokerList) + val channelType = ChannelType.getChannelType(options.valueOf(channelTypeOpt)) val sync = options.has(syncOpt) val compressionCodecOptionValue = options.valueOf(compressionCodecOpt) val compressionCodec = if (options.has(compressionCodecOpt)) diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index d1e7c43..9cf56cd 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -19,18 +19,17 @@ package kafka.tools import joptsimple._ -import org.I0Itec.zkclient.ZkClient -import kafka.utils._ -import kafka.consumer.SimpleConsumer -import kafka.api.{OffsetFetchResponse, OffsetFetchRequest, OffsetRequest} -import kafka.common.{OffsetMetadataAndError, ErrorMapping, BrokerNotAvailableException, TopicAndPartition} -import scala.collection._ +import kafka.api.{OffsetFetchRequest, OffsetFetchResponse, OffsetRequest, PartitionOffsetRequestInfo} import kafka.client.ClientUtils -import kafka.network.BlockingChannel -import kafka.api.PartitionOffsetRequestInfo -import scala.Some +import kafka.common.{BrokerNotAvailableException, ErrorMapping, OffsetMetadataAndError, TopicAndPartition} +import kafka.consumer.SimpleConsumer +import kafka.network.{BlockingChannel, ChannelType} +import kafka.utils._ +import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.exception.ZkNoNodeException +import scala.collection._ + object ConsumerOffsetChecker extends Logging { private val consumerMap: mutable.Map[Int, Option[SimpleConsumer]] = mutable.Map() @@ -111,6 +110,8 @@ object ConsumerOffsetChecker extends Logging { val zkConnectOpt = parser.accepts("zookeeper", "ZooKeeper connect string."). withRequiredArg().defaultsTo("localhost:2181").ofType(classOf[String]) + val channelTypeOpt = parser.accepts("channel", "plaintext or SSL."). + withRequiredArg().defaultsTo("plaintext").ofType(classOf[String]) val topicsOpt = parser.accepts("topic", "Comma-separated list of consumer topics (all topics if absent)."). withRequiredArg().ofType(classOf[String]) @@ -137,6 +138,7 @@ object ConsumerOffsetChecker extends Logging { CommandLineUtils.checkRequiredArgs(parser, options, groupOpt, zkConnectOpt) val zkConnect = options.valueOf(zkConnectOpt) + val channelType = options.valueOf(channelTypeOpt) val group = options.valueOf(groupOpt) val groupDirs = new ZKGroupDirs(group) @@ -158,7 +160,7 @@ object ConsumerOffsetChecker extends Logging { topicPidMap = immutable.Map(ZkUtils.getPartitionsForTopics(zkClient, topicList).toSeq:_*) val topicPartitions = topicPidMap.flatMap { case(topic, partitionSeq) => partitionSeq.map(TopicAndPartition(topic, _)) }.toSeq - val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs) + val channel = ClientUtils.channelToOffsetManager(group, ChannelType.getChannelType(channelType), zkClient, channelSocketTimeoutMs, channelRetryBackoffMs) debug("Sending offset fetch request to coordinator %s:%d.".format(channel.host, channel.port)) channel.send(OffsetFetchRequest(group, topicPartitions)) -- 1.8.4.msysgit.0 From 0c5c51a99a170a42d89a4c314a4692ba0a6593f8 Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Mon, 20 Oct 2014 21:14:41 +0300 Subject: [PATCH 06/12] working --- core/src/main/scala/kafka/tools/ConsoleConsumer.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 323fc85..5c62689 100644 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -41,6 +41,10 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("topic") .ofType(classOf[String]) + val channelTypeOpt = parser.accepts("channel", "plaintext or SSL.") + .withRequiredArg + .describedAs("channel") + .ofType(classOf[String]) val whitelistOpt = parser.accepts("whitelist", "Whitelist of topics to include for consumption.") .withRequiredArg .describedAs("whitelist") @@ -94,6 +98,7 @@ object ConsoleConsumer extends Logging { if (topicOrFilterOpt.size != 1) CommandLineUtils.printUsageAndDie(parser, "Exactly one of whitelist/blacklist/topic is required.") val topicArg = options.valueOf(topicOrFilterOpt.head) + val channelType = options.valueOf(channelTypeOpt) val filterSpec = if (options.has(blacklistOpt)) new Blacklist(topicArg) else @@ -138,6 +143,8 @@ object ConsoleConsumer extends Logging { System.exit(1) } + consumerProps.put("channel", channelType) + if(options.has(deleteConsumerOffsetsOpt)) ZkUtils.maybeDeletePath(options.valueOf(zkConnectOpt), "/consumers/" + consumerProps.getProperty("group.id")) -- 1.8.4.msysgit.0 From 9577f5f27337ed2114d45ce4c265cbcbf63f1f8b Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Tue, 21 Oct 2014 16:32:37 +0300 Subject: [PATCH 07/12] almost done --- .../src/main/java/kafka/etl/KafkaETLContext.java | 3 ++- core/src/main/scala/kafka/client/ClientUtils.scala | 3 ++- .../scala/kafka/consumer/ConsumerFetcherManager.scala | 1 + .../scala/kafka/consumer/ConsumerFetcherThread.scala | 1 + .../src/main/scala/kafka/consumer/SimpleConsumer.scala | 3 ++- .../scala/kafka/javaapi/consumer/SimpleConsumer.scala | 4 +++- .../src/main/scala/kafka/network/BlockingChannel.scala | 8 ++++++-- core/src/main/scala/kafka/network/ChannelFactory.scala | 1 + .../scala/kafka/network/ssl/SSLChannelFactory.scala | 3 ++- .../scala/kafka/server/AbstractFetcherThread.scala | 5 +++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 4 ++-- .../main/scala/kafka/server/ReplicaFetcherThread.scala | 1 + core/src/main/scala/kafka/tools/ConsoleConsumer.scala | 1 + .../main/scala/kafka/tools/ConsumerOffsetChecker.scala | 13 +++++++------ core/src/main/scala/kafka/tools/GetOffsetShell.scala | 11 +++++++++-- .../scala/kafka/tools/ReplicaVerificationTool.scala | 18 ++++++++++++++---- .../scala/kafka/tools/SimpleConsumerPerformance.scala | 3 ++- .../main/scala/kafka/tools/SimpleConsumerShell.scala | 13 ++++++++++--- .../src/main/scala/kafka/tools/UpdateOffsetsInZK.scala | 3 ++- .../main/java/kafka/examples/SimpleConsumerDemo.java | 4 ++++ 20 files changed, 75 insertions(+), 28 deletions(-) diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java index 1d0e0a9..7a8ffe8 100644 --- a/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java +++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/KafkaETLContext.java @@ -33,6 +33,7 @@ import kafka.javaapi.OffsetRequest; import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.message.ByteBufferMessageSet; import kafka.message.MessageAndOffset; +import kafka.network.PlaintextChannelType$; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; @@ -109,7 +110,7 @@ public class KafkaETLContext { // read data from queue URI uri = _request.getURI(); - _consumer = new SimpleConsumer(uri.getHost(), uri.getPort(), _timeout, _bufferSize, "KafkaETLContext"); + _consumer = new SimpleConsumer(uri.getHost(), uri.getPort(), PlaintextChannelType$.MODULE$, _timeout, _bufferSize, "KafkaETLContext"); // get available offset range _offsetRange = getOffsetRange(); diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 3ea1d01..4990fa1 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -85,12 +85,13 @@ object ClientUtils extends Logging{ * @param clientId The client's identifier * @return topic metadata response */ - def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], clientId: String, timeoutMs: Int, + def fetchTopicMetadata(topics: Set[String], brokers: Seq[Broker], channelType: ChannelType, clientId: String, timeoutMs: Int, correlationId: Int = 0): TopicMetadataResponse = { val props = new Properties() props.put("metadata.broker.list", brokers.map(_.connectionString).mkString(",")) props.put("client.id", clientId) props.put("request.timeout.ms", timeoutMs.toString) + props.put("channel", channelType.name) val producerConfig = new ProducerConfig(props) fetchTopicMetadata(topics, brokers, producerConfig, correlationId) } diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index b9e2bea..7bd1682 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -65,6 +65,7 @@ class ConsumerFetcherManager(private val consumerIdString: String, val brokers = getAllBrokersInCluster(zkClient) val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, brokers, + config.channelType, config.clientId, config.socketTimeoutMs, correlationId.getAndIncrement).topicsMetadata diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index f8c1b4e..55f9513 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -32,6 +32,7 @@ class ConsumerFetcherThread(name: String, extends AbstractFetcherThread(name = name, clientId = config.clientId + "-" + name, sourceBroker = sourceBroker, + channelType = config.channelType, socketTimeout = config.socketTimeoutMs, socketBufferSize = config.socketReceiveBufferBytes, fetchSize = config.fetchMessageMaxBytes, diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala index 4f96767..edf1b4f 100644 --- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -29,13 +29,14 @@ import org.apache.kafka.common.utils.Utils._ @threadsafe class SimpleConsumer(val host: String, val port: Int, + val channelType: ChannelType, val soTimeout: Int, val bufferSize: Int, val clientId: String) extends Logging { ConsumerConfig.validateClientId(clientId) private val lock = new Object() - private val blockingChannel = new BlockingChannel(host, port, PlaintextChannelType, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout) + private val blockingChannel = new BlockingChannel(host, port, channelType, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout) val brokerInfo = "host_%s-port_%s".format(host, port) private val fetchRequestAndResponseStats = FetchRequestAndResponseStatsRegistry.getFetchRequestAndResponseStats(clientId) private var isClosed = false diff --git a/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala index 0ab0195..80879e3 100644 --- a/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala +++ b/core/src/main/scala/kafka/javaapi/consumer/SimpleConsumer.scala @@ -17,6 +17,7 @@ package kafka.javaapi.consumer +import kafka.network.ChannelType import kafka.utils.threadsafe import kafka.javaapi.FetchResponse import kafka.javaapi.OffsetRequest @@ -27,11 +28,12 @@ import kafka.javaapi.OffsetRequest @threadsafe class SimpleConsumer(val host: String, val port: Int, + val channelType: ChannelType, val soTimeout: Int, val bufferSize: Int, val clientId: String) { - private val underlying = new kafka.consumer.SimpleConsumer(host, port, soTimeout, bufferSize, clientId) + private val underlying = new kafka.consumer.SimpleConsumer(host, port, channelType, soTimeout, bufferSize, clientId) /** * Fetch a set of messages from a topic. This version of the fetch method diff --git a/core/src/main/scala/kafka/network/BlockingChannel.scala b/core/src/main/scala/kafka/network/BlockingChannel.scala index 1de922c..a9bbe23 100644 --- a/core/src/main/scala/kafka/network/BlockingChannel.scala +++ b/core/src/main/scala/kafka/network/BlockingChannel.scala @@ -47,11 +47,12 @@ class BlockingChannel(val host: String, def connect() = lock synchronized { if (!connected) { try { + warn("Trying to connect to %s:%d via %s channel".format(host, port, channelType.name)) channel = channelType.factory.createClientChannel(host, port, readBufferSize, writeBufferSize, readTimeoutMs) channel.connect(new InetSocketAddress(host, port)) writeChannel = channel - readChannel = Channels.newChannel(channel.socket().getInputStream) + readChannel = Channels.newChannel(Channels.newInputStream(channel)) connected = true // settings may not match what we requested above val msg = "Created socket with SO_TIMEOUT = %d (requested %d), SO_RCVBUF = %d (requested %d), SO_SNDBUF = %d (requested %d)." @@ -62,7 +63,10 @@ class BlockingChannel(val host: String, channel.socket.getSendBufferSize, writeBufferSize)) } catch { - case e: Throwable => disconnect() + case e: Throwable => { + warn(e) + disconnect() + } } } } diff --git a/core/src/main/scala/kafka/network/ChannelFactory.scala b/core/src/main/scala/kafka/network/ChannelFactory.scala index 601468f..3ba2b32 100644 --- a/core/src/main/scala/kafka/network/ChannelFactory.scala +++ b/core/src/main/scala/kafka/network/ChannelFactory.scala @@ -50,6 +50,7 @@ abstract class ChannelFactory extends Logging { } protected def configureClientChannel(channel: SocketChannel, readBufferSize: Int, writeBufferSize: Int, readTimeoutMs: Int) = { + info("Configuring plaintext client channel") if (readBufferSize > 0) channel.socket.setReceiveBufferSize(readBufferSize) if (writeBufferSize > 0) diff --git a/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala b/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala index 853f4a4..3a7a4b7 100644 --- a/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala +++ b/core/src/main/scala/kafka/network/ssl/SSLChannelFactory.scala @@ -36,12 +36,13 @@ object SSLChannelFactory extends ChannelFactory { } override protected def configureClientChannel(channel: SocketChannel, readBufferSize: Int, writeBufferSize: Int, readTimeoutMs: Int): Unit = { + info("Configuring secure client channel") val secureChannel = channel.asInstanceOf[SSLSocketChannel] if (readBufferSize > 0) secureChannel.socket.setReceiveBufferSize(readBufferSize) if (writeBufferSize > 0) secureChannel.socket.setSendBufferSize(writeBufferSize) - secureChannel.simulateBlocking(true) + secureChannel.simulateBlocking(b = true) secureChannel.socket.setSoTimeout(readTimeoutMs) secureChannel.socket.setKeepAlive(true) secureChannel.socket.setTcpNoDelay(true) diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala index 2e9532e..6e89d85 100644 --- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -18,6 +18,7 @@ package kafka.server import kafka.cluster.Broker +import kafka.network.ChannelType import kafka.utils.{Pool, ShutdownableThread} import kafka.consumer.{PartitionTopicInfo, SimpleConsumer} import kafka.api.{FetchRequest, FetchResponse, FetchResponsePartitionData, FetchRequestBuilder} @@ -38,14 +39,14 @@ import com.yammer.metrics.core.Gauge /** * Abstract class for fetching data from multiple partitions from the same broker. */ -abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: Broker, socketTimeout: Int, socketBufferSize: Int, +abstract class AbstractFetcherThread(name: String, clientId: String, sourceBroker: Broker, channelType: ChannelType, socketTimeout: Int, socketBufferSize: Int, fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1, isInterruptible: Boolean = true) extends ShutdownableThread(name, isInterruptible) { private val partitionMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> offset map private val partitionMapLock = new ReentrantLock private val partitionMapCond = partitionMapLock.newCondition() - val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, clientId) + val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, channelType, socketTimeout, socketBufferSize, clientId) private val brokerInfo = "host_%s-port_%s".format(sourceBroker.host, sourceBroker.port) private val metricId = new ClientIdAndBroker(clientId, brokerInfo) val fetcherStats = new FetcherStats(metricId) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index f94ab53..614e780 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -104,7 +104,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the port to publish to ZooKeeper for clients to use. In IaaS environments, this may * need to be different from the port to which the broker binds. If this is not set, * it will publish the same port that the broker binds to. */ - val advertisedPort: Int = props.getInt("advertised.port", port) + val advertisedPort: Int = props.getInt("advertised.port", portsToChannelTypes.head._1) /* the SO_SNDBUFF buffer of the socket sever sockets */ val socketSendBufferBytes: Int = props.getInt("socket.send.buffer.bytes", 100*1024) @@ -328,7 +328,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off */ val deleteTopicEnable = props.getBoolean("delete.topic.enable", false) - val portsToChannelTypes: Map[Int, ChannelType] = { + def portsToChannelTypes: Map[Int, ChannelType] = { val portToChannelType = mutable.Map.empty[Int, ChannelType] if (port > 0) { portToChannelType += port -> ChannelType.getChannelType("plaintext") diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 6879e73..fa77b56 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -31,6 +31,7 @@ class ReplicaFetcherThread(name:String, extends AbstractFetcherThread(name = name, clientId = name, sourceBroker = sourceBroker, + channelType = brokerConfig.portsToChannelTypes.head._2, socketTimeout = brokerConfig.replicaSocketTimeoutMs, socketBufferSize = brokerConfig.replicaSocketReceiveBufferBytes, fetchSize = brokerConfig.replicaFetchMaxBytes, diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 5c62689..e7c993b 100644 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -43,6 +43,7 @@ object ConsoleConsumer extends Logging { .ofType(classOf[String]) val channelTypeOpt = parser.accepts("channel", "plaintext or SSL.") .withRequiredArg + .defaultsTo("plaintext") .describedAs("channel") .ofType(classOf[String]) val whitelistOpt = parser.accepts("whitelist", "Whitelist of topics to include for consumption.") diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala index 9cf56cd..8ef6697 100644 --- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala +++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala @@ -36,7 +36,7 @@ object ConsumerOffsetChecker extends Logging { private val offsetMap: mutable.Map[TopicAndPartition, Long] = mutable.Map() private var topicPidMap: immutable.Map[String, Seq[Int]] = immutable.Map() - private def getConsumer(zkClient: ZkClient, bid: Int): Option[SimpleConsumer] = { + private def getConsumer(zkClient: ZkClient, channelType: ChannelType, bid: Int): Option[SimpleConsumer] = { try { ZkUtils.readDataMaybeNull(zkClient, ZkUtils.BrokerIdsPath + "/" + bid)._1 match { case Some(brokerInfoString) => @@ -45,7 +45,7 @@ object ConsumerOffsetChecker extends Logging { val brokerInfo = m.asInstanceOf[Map[String, Any]] val host = brokerInfo.get("host").get.asInstanceOf[String] val port = brokerInfo.get("port").get.asInstanceOf[Int] - Some(new SimpleConsumer(host, port, 10000, 100000, "ConsumerOffsetChecker")) + Some(new SimpleConsumer(host, port, channelType, 10000, 100000, "ConsumerOffsetChecker")) case None => throw new BrokerNotAvailableException("Broker id %d does not exist".format(bid)) } @@ -60,6 +60,7 @@ object ConsumerOffsetChecker extends Logging { } private def processPartition(zkClient: ZkClient, + channelType: ChannelType, group: String, topic: String, pid: Int) { val topicPartition = TopicAndPartition(topic, pid) val offsetOpt = offsetMap.get(topicPartition) @@ -67,7 +68,7 @@ object ConsumerOffsetChecker extends Logging { val owner = ZkUtils.readDataMaybeNull(zkClient, groupDirs.consumerOwnerDir + "/%s".format(pid))._1 ZkUtils.getLeaderForPartition(zkClient, topic, pid) match { case Some(bid) => - val consumerOpt = consumerMap.getOrElseUpdate(bid, getConsumer(zkClient, bid)) + val consumerOpt = consumerMap.getOrElseUpdate(bid, getConsumer(zkClient, channelType, bid)) consumerOpt match { case Some(consumer) => val topicAndPartition = TopicAndPartition(topic, pid) @@ -85,11 +86,11 @@ object ConsumerOffsetChecker extends Logging { } } - private def processTopic(zkClient: ZkClient, group: String, topic: String) { + private def processTopic(zkClient: ZkClient, channelType: ChannelType, group: String, topic: String) { topicPidMap.get(topic) match { case Some(pids) => pids.sorted.foreach { - pid => processPartition(zkClient, group, topic, pid) + pid => processPartition(zkClient, channelType, group, topic, pid) } case None => // ignore } @@ -193,7 +194,7 @@ object ConsumerOffsetChecker extends Logging { println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format("Group", "Topic", "Pid", "Offset", "logSize", "Lag", "Owner")) topicList.sorted.foreach { - topic => processTopic(zkClient, group, topic) + topic => processTopic(zkClient, ChannelType.getChannelType(channelType), group, topic) } if (options.has("broker-info")) diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 3d9293e..c73482d 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -23,6 +23,7 @@ import joptsimple._ import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} import kafka.common.TopicAndPartition import kafka.client.ClientUtils +import kafka.network.ChannelType import kafka.utils.{ToolsUtils, CommandLineUtils} @@ -34,6 +35,11 @@ object GetOffsetShell { .withRequiredArg .describedAs("hostname:port,...,hostname:port") .ofType(classOf[String]) + val channelTypeOpt = parser.accepts("channel", "plaintext or SSL.") + .withRequiredArg + .defaultsTo("plaintext") + .describedAs("ssl, plaintext") + .ofType(classOf[String]) val topicOpt = parser.accepts("topic", "REQUIRED: The topic to get offset from.") .withRequiredArg .describedAs("topic") @@ -69,13 +75,14 @@ object GetOffsetShell { val brokerList = options.valueOf(brokerListOpt) ToolsUtils.validatePortOrDie(parser, brokerList) val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) + val channelType = options.valueOf(channelTypeOpt) val topic = options.valueOf(topicOpt) var partitionList = options.valueOf(partitionOpt) var time = options.valueOf(timeOpt).longValue val nOffsets = options.valueOf(nOffsetsOpt).intValue val maxWaitMs = options.valueOf(maxWaitMsOpt).intValue() - val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, clientId, maxWaitMs).topicsMetadata + val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, ChannelType.getChannelType(channelType), clientId, maxWaitMs).topicsMetadata if(topicsMetadata.size != 1 || !topicsMetadata(0).topic.equals(topic)) { System.err.println(("Error: no valid topic metadata for topic: %s, " + " probably the topic does not exist, run ").format(topic) + "kafka-list-topic.sh to verify") @@ -93,7 +100,7 @@ object GetOffsetShell { case Some(metadata) => metadata.leader match { case Some(leader) => - val consumer = new SimpleConsumer(leader.host, leader.port, 10000, 100000, clientId) + val consumer = new SimpleConsumer(leader.host, leader.port, ChannelType.getChannelType(channelType), 10000, 100000, clientId) val topicAndPartition = TopicAndPartition(topic, partitionId) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(time, nOffsets))) val offsets = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index ba6ddd7..7edc686 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -28,6 +28,7 @@ import kafka.api._ import java.text.SimpleDateFormat import java.util.Date import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.network.ChannelType import kafka.utils._ import kafka.consumer.{ConsumerConfig, Whitelist, SimpleConsumer} @@ -67,6 +68,11 @@ object ReplicaVerificationTool extends Logging { .withRequiredArg .describedAs("hostname:port,...,hostname:port") .ofType(classOf[String]) + val channelTypeOpt = parser.accepts("channel", "plaintext or SSL.") + .withRequiredArg + .defaultsTo("plaintext") + .describedAs("ssl, plaintext") + .ofType(classOf[String]) val fetchSizeOpt = parser.accepts("fetch-size", "The fetch size of each request.") .withRequiredArg .describedAs("bytes") @@ -119,7 +125,8 @@ object ReplicaVerificationTool extends Logging { val brokerList = options.valueOf(brokerListOpt) ToolsUtils.validatePortOrDie(parser,brokerList) val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) - val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, clientId, maxWaitMs) + val channelType = options.valueOf(channelTypeOpt) + val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, ChannelType.getChannelType(channelType), clientId, maxWaitMs) val brokerMap = topicsMetadataResponse.brokers.map(b => (b.id, b)).toMap val filteredTopicMetadata = topicsMetadataResponse.topicsMetadata.filter( topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic, excludeInternalTopics = false)) @@ -156,6 +163,7 @@ object ReplicaVerificationTool extends Logging { val replicaBuffer = new ReplicaBuffer(expectedReplicasPerTopicAndPartition, leadersPerBroker, + ChannelType.getChannelType(channelType), topicAndPartitionsPerBroker.size, brokerMap, initialOffsetTime, @@ -166,6 +174,7 @@ object ReplicaVerificationTool extends Logging { case (brokerId, topicAndPartitions) => new ReplicaFetcher(name = "ReplicaFetcher-" + brokerId, sourceBroker = brokerMap(brokerId), + ChannelType.getChannelType(channelType), topicAndPartitions = topicAndPartitions, replicaBuffer = replicaBuffer, socketTimeout = 30000, @@ -196,6 +205,7 @@ private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, c private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int], leadersPerBroker: Map[Int, Seq[TopicAndPartition]], + channelType: ChannelType, expectedNumFetchers: Int, brokerMap: Map[Int, Broker], initialOffsetTime: Long, @@ -237,7 +247,7 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa private def setInitialOffsets() { for ((brokerId, topicAndPartitions) <- leadersPerBroker) { val broker = brokerMap(brokerId) - val consumer = new SimpleConsumer(broker.host, broker.port, 10000, 100000, ReplicaVerificationTool.clientId) + val consumer = new SimpleConsumer(broker.host, broker.port, channelType, 10000, 100000, ReplicaVerificationTool.clientId) val initialOffsetMap: Map[TopicAndPartition, PartitionOffsetRequestInfo] = topicAndPartitions.map(topicAndPartition => topicAndPartition -> PartitionOffsetRequestInfo(initialOffsetTime, 1)).toMap val offsetRequest = OffsetRequest(initialOffsetMap) @@ -335,11 +345,11 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa } } -private class ReplicaFetcher(name: String, sourceBroker: Broker, topicAndPartitions: Iterable[TopicAndPartition], +private class ReplicaFetcher(name: String, sourceBroker: Broker, channelType: ChannelType, topicAndPartitions: Iterable[TopicAndPartition], replicaBuffer: ReplicaBuffer, socketTimeout: Int, socketBufferSize: Int, fetchSize: Int, maxWait: Int, minBytes: Int, doVerification: Boolean) extends ShutdownableThread(name) { - val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize, ReplicaVerificationTool.clientId) + val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, channelType, socketTimeout, socketBufferSize, ReplicaVerificationTool.clientId) val fetchRequestBuilder = new FetchRequestBuilder(). clientId(ReplicaVerificationTool.clientId). replicaId(Request.DebuggingConsumerId). diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala index 7602b8d..adf3214 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala @@ -21,6 +21,7 @@ import java.net.URI import java.text.SimpleDateFormat import kafka.api.{PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest} import kafka.consumer.SimpleConsumer +import kafka.network.PlaintextChannelType import kafka.utils._ import org.apache.log4j.Logger import kafka.common.TopicAndPartition @@ -42,7 +43,7 @@ object SimpleConsumerPerformance { println("time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec") } - val consumer = new SimpleConsumer(config.url.getHost, config.url.getPort, 30*1000, 2*config.fetchSize, config.clientId) + val consumer = new SimpleConsumer(config.url.getHost, config.url.getPort, PlaintextChannelType, 30*1000, 2*config.fetchSize, config.clientId) // reset to latest or smallest offset val topicAndPartition = TopicAndPartition(config.topic, config.partition) diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index b4f903b..0c86945 100644 --- a/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ b/core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -18,6 +18,7 @@ package kafka.tools import joptsimple._ +import kafka.network.ChannelType import kafka.utils._ import kafka.consumer._ import kafka.client.ClientUtils @@ -40,6 +41,11 @@ object SimpleConsumerShell extends Logging { .withRequiredArg .describedAs("hostname:port,...,hostname:port") .ofType(classOf[String]) + val channelTypeOpt = parser.accepts("channel", "plaintext or SSL.") + .withRequiredArg + .defaultsTo("plaintext") + .describedAs("ssl, plaintext") + .ofType(classOf[String]) val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.") .withRequiredArg .describedAs("topic") @@ -127,7 +133,8 @@ object SimpleConsumerShell extends Logging { val brokerList = options.valueOf(brokerListOpt) ToolsUtils.validatePortOrDie(parser,brokerList) val metadataTargetBrokers = ClientUtils.parseBrokerList(brokerList) - val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, clientId, maxWaitMs).topicsMetadata + val channelType = ChannelType.getChannelType(options.valueOf(channelTypeOpt)) + val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), metadataTargetBrokers, channelType, clientId, maxWaitMs).topicsMetadata if(topicsMetadata.size != 1 || !topicsMetadata(0).topic.equals(topic)) { System.err.println(("Error: no valid topic metadata for topic: %s, " + "what we get from server is only: %s").format(topic, topicsMetadata)) System.exit(1) @@ -167,7 +174,7 @@ object SimpleConsumerShell extends Logging { System.exit(1) } if (startingOffset < 0) { - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, ConsumerConfig.SocketTimeout, + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, channelType, ConsumerConfig.SocketTimeout, ConsumerConfig.SocketBufferSize, clientId) try { startingOffset = simpleConsumer.earliestOrLatestOffset(TopicAndPartition(topic, partitionId), startingOffset, @@ -189,7 +196,7 @@ object SimpleConsumerShell extends Logging { val replicaString = if(replicaId > 0) "leader" else "replica" info("Starting simple consumer shell to partition [%s, %d], %s [%d], host and port: [%s, %d], from offset [%d]" .format(topic, partitionId, replicaString, replicaId, fetchTargetBroker.host, fetchTargetBroker.port, startingOffset)) - val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, 10000, 64*1024, clientId) + val simpleConsumer = new SimpleConsumer(fetchTargetBroker.host, fetchTargetBroker.port, channelType, 10000, 64*1024, clientId) val thread = Utils.newThread("kafka-simpleconsumer-shell", new Runnable() { def run() { var offset = startingOffset diff --git a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala index 111c9a8..092891a 100644 --- a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala +++ b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala @@ -17,6 +17,7 @@ package kafka.tools +import kafka.network.PlaintextChannelType import org.I0Itec.zkclient.ZkClient import kafka.consumer.{SimpleConsumer, ConsumerConfig} import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} @@ -65,7 +66,7 @@ object UpdateOffsetsInZK { ZkUtils.getBrokerInfo(zkClient, broker) match { case Some(brokerInfo) => - val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024, "UpdateOffsetsInZk") + val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, PlaintextChannelType, 10000, 100 * 1024, "UpdateOffsetsInZk") val topicAndPartition = TopicAndPartition(topic, partition) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1))) val offset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head diff --git a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java index c79192c..caae937 100644 --- a/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java +++ b/examples/src/main/java/kafka/examples/SimpleConsumerDemo.java @@ -27,6 +27,9 @@ import java.util.List; import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.message.ByteBufferMessageSet; import kafka.message.MessageAndOffset; +import kafka.network.PlaintextChannelType; +import kafka.network.PlaintextChannelType$; + import java.util.HashMap; import java.util.Map; @@ -58,6 +61,7 @@ public class SimpleConsumerDemo { SimpleConsumer simpleConsumer = new SimpleConsumer(KafkaProperties.kafkaServerURL, KafkaProperties.kafkaServerPort, + PlaintextChannelType$.MODULE$, KafkaProperties.connectionTimeOut, KafkaProperties.kafkaProducerBufferSize, KafkaProperties.clientId); -- 1.8.4.msysgit.0 From d9a6d537eb69bcfff1c72b1bac06afb7ca4fd158 Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Tue, 21 Oct 2014 21:18:00 +0300 Subject: [PATCH 08/12] almost done --- core/src/main/scala/kafka/client/ClientUtils.scala | 6 ++--- .../kafka/consumer/ConsumerFetcherManager.scala | 26 +++++++++++++--------- core/src/main/scala/kafka/utils/ZkUtils.scala | 15 +++++++++++++ 3 files changed, 33 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 4990fa1..af6e7b2 100644 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -116,9 +116,9 @@ object ClientUtils extends Logging{ while (!connected) { val allBrokers = getAllBrokersInCluster(zkClient) Random.shuffle(allBrokers).find { broker => - val channels = getBrokerChannels(zkClient, broker.id).filter(_.channelType == channelType) - if (channels.nonEmpty) { - val channelInfo = channels.head + val channelInfoOpt = getExpectedChannelForBroker(zkClient, broker.id, channelType) + if (channelInfoOpt.isDefined) { + val channelInfo = channelInfoOpt.get trace("Connecting to broker %s:%d.".format(broker.host, channelInfo.port)) try { channel = new BlockingChannel(broker.host, channelInfo.port, channelInfo.channelType, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, socketTimeoutMs) diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala index 7bd1682..e710f25 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala @@ -17,20 +17,20 @@ package kafka.consumer -import org.I0Itec.zkclient.ZkClient -import kafka.server.{BrokerAndInitialOffset, AbstractFetcherThread, AbstractFetcherManager} -import kafka.cluster.{Cluster, Broker} -import scala.collection.immutable -import scala.collection.Map -import collection.mutable.HashMap -import scala.collection.mutable +import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.locks.ReentrantLock + +import kafka.client.ClientUtils +import kafka.cluster.{Broker, Cluster} +import kafka.common.{BrokerChannelNotAvailableException, TopicAndPartition} +import kafka.server.{AbstractFetcherManager, AbstractFetcherThread, BrokerAndInitialOffset} import kafka.utils.Utils.inLock import kafka.utils.ZkUtils._ import kafka.utils.{ShutdownableThread, SystemTime} -import kafka.common.TopicAndPartition -import kafka.client.ClientUtils -import java.util.concurrent.atomic.AtomicInteger +import org.I0Itec.zkclient.ZkClient + +import scala.collection.{immutable, mutable} +import scala.collection.mutable.HashMap /** * Usage: @@ -62,7 +62,11 @@ class ConsumerFetcherManager(private val consumerIdString: String, } trace("Partitions without leader %s".format(noLeaderPartitionSet)) - val brokers = getAllBrokersInCluster(zkClient) + val brokers = getBrokersForChannel(zkClient, config.channelType) + + if (brokers.isEmpty) + throw new BrokerChannelNotAvailableException("There is no broker available with channel %s".format(config.channelType.name)) + val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet, brokers, config.channelType, diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index d17968a..eec5a8b 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -79,6 +79,17 @@ object ZkUtils extends Logging { brokerIds.map(_.toInt).map(getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) } + def getBrokersForChannel(zkClient: ZkClient, channelType: ChannelType): Seq[Broker] = { + getAllBrokersInCluster(zkClient).map(broker => { + val channel = getExpectedChannelForBroker(zkClient, broker.id, channelType) + if (channel.isDefined) { + Some(new Broker(broker.id, broker.host, channel.get.port)) + } else { + None + } + }).filter(_.isDefined).map(_.get) + } + def getLeaderAndIsrForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderAndIsr] = { ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr) } @@ -714,6 +725,10 @@ object ZkUtils extends Logging { channels.map { channel => ChannelInfo.createChannelInfo(brokerId, channel) } } + def getExpectedChannelForBroker(zkClient: ZkClient, brokerId: Int, channelType: ChannelType): Option[ChannelInfo] = { + getBrokerChannels(zkClient, brokerId).find(_.channelType == channelType) + } + def getAllTopics(zkClient: ZkClient): Seq[String] = { val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath) if(topics == null) -- 1.8.4.msysgit.0 From ed79ed3117234ac7936d95424b5d869b96ff0696 Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Wed, 22 Oct 2014 19:26:39 +0300 Subject: [PATCH 09/12] done --- core/src/main/scala/kafka/api/RequestKeys.scala | 4 +- .../kafka/api/TopicMetadataForChannelRequest.scala | 107 +++++++++++++++++++++ .../scala/kafka/api/UpdateMetadataRequest.scala | 19 +++- .../controller/ControllerChannelManager.scala | 22 +++-- .../scala/kafka/controller/KafkaController.scala | 34 +++---- .../kafka/controller/ReplicaStateMachine.scala | 16 +-- .../src/main/scala/kafka/network/ChannelInfo.scala | 20 ++++ .../main/scala/kafka/producer/SyncProducer.scala | 2 +- core/src/main/scala/kafka/server/KafkaApis.scala | 13 +++ core/src/main/scala/kafka/server/KafkaConfig.scala | 8 +- .../main/scala/kafka/server/MetadataCache.scala | 22 +++++ .../main/scala/kafka/tools/UpdateOffsetsInZK.scala | 10 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 4 +- 13 files changed, 236 insertions(+), 45 deletions(-) create mode 100644 core/src/main/scala/kafka/api/TopicMetadataForChannelRequest.scala diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala index c24c034..04f4a5a 100644 --- a/core/src/main/scala/kafka/api/RequestKeys.scala +++ b/core/src/main/scala/kafka/api/RequestKeys.scala @@ -34,6 +34,7 @@ object RequestKeys { val ConsumerMetadataKey: Short = 10 val JoinGroupKey: Short = 11 val HeartbeatKey: Short = 12 + val MetadataForChannelsKey: Short = 13 val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]= Map(ProduceKey -> ("Produce", ProducerRequest.readFrom), @@ -48,7 +49,8 @@ object RequestKeys { OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom), ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom), JoinGroupKey -> ("JoinGroup", JoinGroupRequestAndHeader.readFrom), - HeartbeatKey -> ("Heartbeat", HeartbeatRequestAndHeader.readFrom) + HeartbeatKey -> ("Heartbeat", HeartbeatRequestAndHeader.readFrom), + MetadataForChannelsKey -> ("MetadataForChannel", TopicMetadataForChannelRequest.readFrom) ) def nameForKey(key: Short): String = { diff --git a/core/src/main/scala/kafka/api/TopicMetadataForChannelRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataForChannelRequest.scala new file mode 100644 index 0000000..047693b --- /dev/null +++ b/core/src/main/scala/kafka/api/TopicMetadataForChannelRequest.scala @@ -0,0 +1,107 @@ +/** + * 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 kafka.api + +import java.nio.ByteBuffer + +import kafka.api.ApiUtils._ +import kafka.common.ErrorMapping +import kafka.network.{ChannelType, BoundedByteBufferSend, RequestChannel} +import kafka.network.RequestChannel.Response +import kafka.utils.Logging + +import scala.collection.mutable.ListBuffer + +object TopicMetadataForChannelRequest extends Logging { + val CurrentVersion = 0.shortValue + val DefaultClientId = "" + + /** + * TopicMetadataWithChannelsRequest has the following format - + * number of topics (4 bytes) list of topics (2 bytes + topic.length per topic) detailedMetadata (2 bytes) timestamp (8 bytes) count (4 bytes) + */ + + def readFrom(buffer: ByteBuffer): TopicMetadataForChannelRequest = { + val versionId = buffer.getShort + val correlationId = buffer.getInt + val clientId = readShortString(buffer) + val numTopics = readIntInRange(buffer, "number of topics", (0, Int.MaxValue)) + val topics = new ListBuffer[String]() + for (i <- 0 until numTopics) + topics += readShortString(buffer) + val channelType = ChannelType.getChannelType(readShortString(buffer)) + new TopicMetadataForChannelRequest(versionId, correlationId, clientId, topics.toList, channelType) + } +} + +case class TopicMetadataForChannelRequest(val versionId: Short, + val correlationId: Int, + val clientId: String, + val topics: Seq[String], + val channelType: ChannelType) + extends RequestOrResponse(Some(RequestKeys.MetadataForChannelsKey)) { + + def this(request: TopicMetadataRequest, channelType: ChannelType) = { + this(request.versionId, request.correlationId, request.clientId, request.topics, channelType) + } + + def this(topics: Seq[String], correlationId: Int, channelType: ChannelType) = + this(TopicMetadataForChannelRequest.CurrentVersion, correlationId, TopicMetadataForChannelRequest.DefaultClientId, topics, channelType) + + def writeTo(buffer: ByteBuffer) { + buffer.putShort(versionId) + buffer.putInt(correlationId) + writeShortString(buffer, clientId) + buffer.putInt(topics.size) + topics.foreach(topic => writeShortString(buffer, topic)) + writeShortString(buffer, channelType.name) + } + + def sizeInBytes(): Int = { + 2 + /* version id */ + 4 + /* correlation id */ + shortStringLength(clientId) + /* client id */ + 4 + /* number of topics */ + topics.foldLeft(0)(_ + shortStringLength(_)) + /* topics */ + shortStringLength(channelType.name) + } + + override def toString(): String = { + describe(true) + } + + override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = { + val topicMetadata = topics.map { + topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) + } + val errorResponse = TopicMetadataResponse(Seq(), topicMetadata, correlationId) + requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse))) + } + + override def describe(details: Boolean): String = { + val TopicMetadataWithChannelsRequest = new StringBuilder + TopicMetadataWithChannelsRequest.append("Name: " + this.getClass.getSimpleName) + TopicMetadataWithChannelsRequest.append("; Version: " + versionId) + TopicMetadataWithChannelsRequest.append("; CorrelationId: " + correlationId) + TopicMetadataWithChannelsRequest.append("; ClientId: " + clientId) + if (details) + TopicMetadataWithChannelsRequest.append("; Topics: " + topics.mkString(",")) + TopicMetadataWithChannelsRequest.append("; ChannelType: " + channelType.name) + TopicMetadataWithChannelsRequest.toString() + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala index 530982e..8c5f6c1 100644 --- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala @@ -20,7 +20,7 @@ import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.cluster.Broker import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.network.{BoundedByteBufferSend, RequestChannel} +import kafka.network.{ChannelInfo, BoundedByteBufferSend, RequestChannel} import kafka.network.RequestChannel.Response import collection.Set @@ -49,8 +49,10 @@ object UpdateMetadataRequest { val numAliveBrokers = buffer.getInt val aliveBrokers = for(i <- 0 until numAliveBrokers) yield Broker.readFrom(buffer) + val numAliveBrokerChannels = buffer.getInt + val aliveBrokerChannels = for(i <- 0 until numAliveBrokerChannels) yield ChannelInfo.readFrom(buffer) new UpdateMetadataRequest(versionId, correlationId, clientId, controllerId, controllerEpoch, - partitionStateInfos.toMap, aliveBrokers.toSet) + partitionStateInfos.toMap, aliveBrokers.toSet, aliveBrokerChannels) } } @@ -60,13 +62,14 @@ case class UpdateMetadataRequest (versionId: Short, controllerId: Int, controllerEpoch: Int, partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], - aliveBrokers: Set[Broker]) + aliveBrokers: Set[Broker], + aliveBrokerChannels: Seq[ChannelInfo]) extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey)) { def this(controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String, - partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker]) = { + partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker], aliveBrokerChannels: Seq[ChannelInfo]) = { this(UpdateMetadataRequest.CurrentVersion, correlationId, clientId, - controllerId, controllerEpoch, partitionStateInfos, aliveBrokers) + controllerId, controllerEpoch, partitionStateInfos, aliveBrokers, aliveBrokerChannels) } def writeTo(buffer: ByteBuffer) { @@ -83,6 +86,8 @@ case class UpdateMetadataRequest (versionId: Short, } buffer.putInt(aliveBrokers.size) aliveBrokers.foreach(_.writeTo(buffer)) + buffer.putInt(aliveBrokerChannels.size) + aliveBrokerChannels.foreach(_.writeTo(buffer)) } def sizeInBytes(): Int = { @@ -98,6 +103,9 @@ case class UpdateMetadataRequest (versionId: Short, size += 4 /* number of alive brokers in the cluster */ for(broker <- aliveBrokers) size += broker.sizeInBytes /* broker info */ + size += 4 /* number of alive brokers channels in the cluster */ + for(channel <- aliveBrokerChannels) + size += channel.sizeInBytes /* broker channels info */ size } @@ -119,6 +127,7 @@ case class UpdateMetadataRequest (versionId: Short, updateMetadataRequest.append(";CorrelationId:" + correlationId) updateMetadataRequest.append(";ClientId:" + clientId) updateMetadataRequest.append(";AliveBrokers:" + aliveBrokers.mkString(",")) + updateMetadataRequest.append(";AliveBrokerChannels:" + aliveBrokerChannels.mkString(",")) if(details) updateMetadataRequest.append(";PartitionState:" + partitionStateInfos.mkString(",")) updateMetadataRequest.toString() diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 5a736db..da348af 100644 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -20,8 +20,8 @@ import java.util.concurrent.{BlockingQueue, LinkedBlockingQueue} import kafka.api.{RequestOrResponse, _} import kafka.cluster.Broker -import kafka.common.TopicAndPartition -import kafka.network.{BlockingChannel, Receive} +import kafka.common.{BrokerChannelNotAvailableException, TopicAndPartition} +import kafka.network.{ChannelInfo, BlockingChannel, Receive} import kafka.server.KafkaConfig import kafka.utils.{Logging, ShutdownableThread, Utils} @@ -33,7 +33,7 @@ class ControllerChannelManager (private val controllerContext: ControllerContext private val brokerLock = new Object this.logIdent = "[Channel manager on controller " + config.brokerId + "]: " - controllerContext.liveBrokers.foreach(addNewBroker(_)) + controllerContext.liveBrokers.foreach(broker => addNewBroker(broker, controllerContext.liveBrokerChannels.get(broker.id).get.toSet)) def startup() = { brokerLock synchronized { @@ -59,11 +59,11 @@ class ControllerChannelManager (private val controllerContext: ControllerContext } } - def addBroker(broker: Broker) { + def addBroker(broker: Broker, channels: Set[ChannelInfo]) { // be careful here. Maybe the startup() API has already started the request send thread brokerLock synchronized { if(!brokerStateInfo.contains(broker.id)) { - addNewBroker(broker) + addNewBroker(broker, channels) startRequestSendThread(broker.id) } } @@ -75,12 +75,15 @@ class ControllerChannelManager (private val controllerContext: ControllerContext } } - private def addNewBroker(broker: Broker) { + private def addNewBroker(broker: Broker, channels: Set[ChannelInfo]) { val messageQueue = new LinkedBlockingQueue[(RequestOrResponse, (RequestOrResponse) => Unit)](config.controllerMessageQueueSize) debug("Controller %d trying to connect to broker %d".format(config.brokerId,broker.id)) + val availableChannels = channels.intersect(config.availableChannels) + if (availableChannels.isEmpty) throw new BrokerChannelNotAvailableException("There are no suitable channels to connect to") + val channel = new BlockingChannel(broker.host, - config.portsToChannelTypes.head._1, - config.portsToChannelTypes.head._2, + availableChannels.head.port, + availableChannels.head.channelType, BlockingChannel.UseDefaultBufferSize, BlockingChannel.UseDefaultBufferSize, config.controllerSocketTimeoutMs) @@ -291,8 +294,9 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging updateMetadataRequestMap.foreach { m => val broker = m._1 val partitionStateInfos = m._2.toMap + val allChannels = controllerContext.liveBrokerChannels.values.flatten.toSeq val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, correlationId, clientId, - partitionStateInfos, controllerContext.liveOrShuttingDownBrokers) + partitionStateInfos, controllerContext.liveOrShuttingDownBrokers, allChannels) partitionStateInfos.foreach(p => stateChangeLogger.trace(("Controller %d epoch %d sending UpdateMetadata request %s with " + "correlationId %d to broker %d for partition %s").format(controllerId, controllerEpoch, p._2.leaderIsrAndControllerEpoch, correlationId, broker, p._1))) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 8ab4a1b..0f96662 100644 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -16,30 +16,27 @@ */ package kafka.controller -import collection._ -import collection.Set -import com.yammer.metrics.core.Gauge -import java.lang.{IllegalStateException, Object} import java.util.concurrent.TimeUnit -import kafka.admin.AdminUtils -import kafka.admin.PreferredReplicaLeaderElectionCommand +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.locks.ReentrantLock + +import com.yammer.metrics.core.Gauge +import kafka.admin.{AdminUtils, PreferredReplicaLeaderElectionCommand} import kafka.api._ import kafka.cluster.Broker -import kafka.common._ +import kafka.common.{TopicAndPartition, _} import kafka.log.LogConfig -import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} +import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} +import kafka.network.ChannelInfo +import kafka.server._ +import kafka.utils.Utils._ import kafka.utils.ZkUtils._ import kafka.utils._ -import kafka.utils.Utils._ -import org.apache.zookeeper.Watcher.Event.KeeperState +import org.I0Itec.zkclient.exception.{ZkNoNodeException, ZkNodeExistsException} import org.I0Itec.zkclient.{IZkDataListener, IZkStateListener, ZkClient} -import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} -import java.util.concurrent.atomic.AtomicInteger -import java.util.concurrent.locks.ReentrantLock -import scala.None -import kafka.server._ -import scala.Some -import kafka.common.TopicAndPartition +import org.apache.zookeeper.Watcher.Event.KeeperState + +import scala.collection.{Set, _} class ControllerContext(val zkClient: ZkClient, val zkSessionTimeout: Int) { @@ -58,16 +55,19 @@ class ControllerContext(val zkClient: ZkClient, private var liveBrokersUnderlying: Set[Broker] = Set.empty private var liveBrokerIdsUnderlying: Set[Int] = Set.empty + private var liveBrokerChannelsUnderlying: Map[Int, Seq[ChannelInfo]] = Map.empty // setter def liveBrokers_=(brokers: Set[Broker]) { liveBrokersUnderlying = brokers liveBrokerIdsUnderlying = liveBrokersUnderlying.map(_.id) + liveBrokerChannelsUnderlying = liveBrokersUnderlying.map(broker => broker.id -> getBrokerChannels(zkClient, broker.id)).toMap } // getter def liveBrokers = liveBrokersUnderlying.filter(broker => !shuttingDownBrokerIds.contains(broker.id)) def liveBrokerIds = liveBrokerIdsUnderlying.filter(brokerId => !shuttingDownBrokerIds.contains(brokerId)) + def liveBrokerChannels = liveBrokerChannelsUnderlying.filter(entry => !shuttingDownBrokerIds.contains(entry._1)) def liveOrShuttingDownBrokerIds = liveBrokerIdsUnderlying def liveOrShuttingDownBrokers = liveBrokersUnderlying diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala index 3e87e1d..6ccfdd2 100644 --- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala +++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala @@ -16,15 +16,16 @@ */ package kafka.controller -import collection._ -import collection.JavaConversions._ import java.util.concurrent.atomic.AtomicBoolean -import kafka.common.{TopicAndPartition, StateChangeFailedException} -import kafka.utils.{ZkUtils, ReplicationUtils, Logging} -import org.I0Itec.zkclient.IZkChildListener -import org.apache.log4j.Logger + +import kafka.common.{StateChangeFailedException, TopicAndPartition} import kafka.controller.Callbacks._ import kafka.utils.Utils._ +import kafka.utils.{Logging, ReplicationUtils, ZkUtils} +import org.I0Itec.zkclient.IZkChildListener + +import scala.collection.JavaConversions._ +import scala.collection._ /** * This class represents the state machine for replicas. It defines the states that a replica can be in, and @@ -360,12 +361,13 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging { val curBrokerIds = currentBrokerList.map(_.toInt).toSet val newBrokerIds = curBrokerIds -- controllerContext.liveOrShuttingDownBrokerIds val newBrokerInfo = newBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _)) + val newBrokerChannels = newBrokerIds.map(id => id -> ZkUtils.getBrokerChannels(zkClient, id).toSet).toMap val newBrokers = newBrokerInfo.filter(_.isDefined).map(_.get) val deadBrokerIds = controllerContext.liveOrShuttingDownBrokerIds -- curBrokerIds controllerContext.liveBrokers = curBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get) info("Newly added brokers: %s, deleted brokers: %s, all live brokers: %s" .format(newBrokerIds.mkString(","), deadBrokerIds.mkString(","), controllerContext.liveBrokerIds.mkString(","))) - newBrokers.foreach(controllerContext.controllerChannelManager.addBroker(_)) + newBrokers.foreach(broker => controllerContext.controllerChannelManager.addBroker(broker, newBrokerChannels.get(broker.id).get)) deadBrokerIds.foreach(controllerContext.controllerChannelManager.removeBroker(_)) if(newBrokerIds.size > 0) controller.onBrokerStartup(newBrokerIds.toSeq) diff --git a/core/src/main/scala/kafka/network/ChannelInfo.scala b/core/src/main/scala/kafka/network/ChannelInfo.scala index 9d11ec1..30eb6bb 100644 --- a/core/src/main/scala/kafka/network/ChannelInfo.scala +++ b/core/src/main/scala/kafka/network/ChannelInfo.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.common.{BrokerChannelNotAvailableException, KafkaException, BrokerNotAvailableException} import kafka.utils.Json +import kafka.utils.Utils._ object ChannelInfo { @@ -53,5 +54,24 @@ object ChannelInfo { } class ChannelInfo(val brokerId: Int, val port: Int, val channelType: ChannelType) { + + def writeTo(buffer: ByteBuffer) { + buffer.putInt(brokerId) + buffer.putInt(port) + writeShortString(buffer, channelType.name) + } + + def sizeInBytes: Int = shortStringLength(channelType.name) /* channel type */ + 4 /* port */ + 4 /* broker id*/ + override def toString: String = "brokerId:%d,port:%d,type:%s".format(brokerId, port, channelType) + + override def equals(obj: Any): Boolean = { + obj match { + case null => false + case n: ChannelInfo => brokerId == n.brokerId && port == n.port && channelType == n.channelType + case _ => false + } + } + + override def hashCode(): Int = hashcode(brokerId, port, channelType) } diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index 60fa44a..42513fe 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -111,7 +111,7 @@ class SyncProducer(val config: SyncProducerConfig) extends Logging { } def send(request: TopicMetadataRequest): TopicMetadataResponse = { - val response = doSend(request) + val response = doSend(new TopicMetadataForChannelRequest(request, config.channelType)) TopicMetadataResponse.readFrom(response.buffer) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index c584b55..e0e0e8a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -60,6 +60,7 @@ class KafkaApis(val requestChannel: RequestChannel, case RequestKeys.FetchKey => handleFetchRequest(request) case RequestKeys.OffsetsKey => handleOffsetRequest(request) case RequestKeys.MetadataKey => handleTopicMetadataRequest(request) + case RequestKeys.MetadataForChannelsKey => handleTopicMetadataForChannelsRequest(request) case RequestKeys.LeaderAndIsrKey => handleLeaderAndIsrRequest(request) case RequestKeys.StopReplicaKey => handleStopReplicaRequest(request) case RequestKeys.UpdateMetadataKey => handleUpdateMetadataRequest(request) @@ -495,6 +496,18 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) } + /** + * Service the topic metadata request API with channels info + */ + def handleTopicMetadataForChannelsRequest(request: RequestChannel.Request) { + val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataForChannelRequest] + val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet) + val brokers = metadataCache.getAliveBrokersForChannel(metadataRequest.channelType) + trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) + val response = new TopicMetadataResponse(brokers, topicMetadata, metadataRequest.correlationId) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } + /* * Service the Offset fetch API */ diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 614e780..e4cb919 100644 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -21,7 +21,7 @@ import java.util.Properties import kafka.consumer.ConsumerConfig import kafka.message.{Message, MessageSet} -import kafka.network.ChannelType +import kafka.network.{ChannelInfo, ChannelType} import kafka.network.ssl.SSLConnectionConfig import kafka.utils.{Utils, VerifiableProperties, ZKConfig} @@ -341,5 +341,9 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro if (portToChannelType.isEmpty) portToChannelType += 9092 -> ChannelType.getChannelType("plaintext") portToChannelType.toMap - } + } + + def availableChannels: Set[ChannelInfo] = { + portsToChannelTypes.map(entry => new ChannelInfo(brokerId, entry._1, entry._2)).toSet + } } diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index bf81a1a..1c76901 100644 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -17,6 +17,8 @@ package kafka.server +import kafka.network.{ChannelType, ChannelInfo} + import scala.collection.{Seq, Set, mutable} import kafka.api._ import kafka.cluster.Broker @@ -34,6 +36,7 @@ private[server] class MetadataCache { private val cache: mutable.Map[String, mutable.Map[Int, PartitionStateInfo]] = new mutable.HashMap[String, mutable.Map[Int, PartitionStateInfo]]() private var aliveBrokers: Map[Int, Broker] = Map() + private var aliveBrokerChannels: Map[Int, List[ChannelInfo]] = Map() private val partitionMetadataLock = new ReentrantReadWriteLock() def getTopicMetadata(topics: Set[String]) = { @@ -86,6 +89,19 @@ private[server] class MetadataCache { } } + def getAliveBrokersForChannel(channelType: ChannelType) = { + inReadLock(partitionMetadataLock) { + aliveBrokers.values.map(broker => { + val channel = aliveBrokerChannels.get(broker.id).get.find(_.channelType == channelType) + if (channel.isDefined) { + Some(new Broker(broker.id, broker.host, channel.get.port)) + } else { + None + } + }).filter(_.isDefined).map(_.get).toSeq + } + } + def addOrUpdatePartitionInfo(topic: String, partitionId: Int, stateInfo: PartitionStateInfo) { @@ -130,6 +146,12 @@ private[server] class MetadataCache { updateMetadataRequest.controllerEpoch, updateMetadataRequest.correlationId)) } } + aliveBrokerChannels = updateMetadataRequest.aliveBrokerChannels.foldLeft(mutable.Map.empty[Int, List[ChannelInfo]])((collection, element) => { + val entry = collection.get(brokerId) + if (entry.isDefined) collection += element.brokerId -> entry.get.::(element) + else collection += element.brokerId -> List(element) + collection + }).toMap } } diff --git a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala index 092891a..b8f2617 100644 --- a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala +++ b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala @@ -21,7 +21,7 @@ import kafka.network.PlaintextChannelType import org.I0Itec.zkclient.ZkClient import kafka.consumer.{SimpleConsumer, ConsumerConfig} import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest} -import kafka.common.{TopicAndPartition, KafkaException} +import kafka.common.{BrokerChannelNotAvailableException, TopicAndPartition, KafkaException} import kafka.utils.{ZKGroupTopicDirs, ZkUtils, ZKStringSerializer, Utils} @@ -64,9 +64,15 @@ object UpdateOffsetsInZK { "getOffsetsBefore request") } + val channelType = PlaintextChannelType ZkUtils.getBrokerInfo(zkClient, broker) match { case Some(brokerInfo) => - val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, PlaintextChannelType, 10000, 100 * 1024, "UpdateOffsetsInZk") + val channelInfo = ZkUtils.getExpectedChannelForBroker(zkClient, brokerInfo.id, channelType) + if (channelInfo.isEmpty) + throw new BrokerChannelNotAvailableException("There is no channel %s available in broker %s".format(channelType.name, + brokerInfo.id)) + + val consumer = new SimpleConsumer(brokerInfo.host, channelInfo.get.port, channelType, 10000, 100 * 1024, "UpdateOffsetsInZk") val topicAndPartition = TopicAndPartition(topic, partition) val request = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(offsetOption, 1))) val offset = consumer.getOffsetsBefore(request).partitionErrorAndOffsets(topicAndPartition).offsets.head diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index eec5a8b..40eebb5 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -722,7 +722,9 @@ object ZkUtils extends Logging { if(channels == null) Seq.empty[ChannelInfo] else - channels.map { channel => ChannelInfo.createChannelInfo(brokerId, channel) } + channels.map { channelName => ChannelInfo.createChannelInfo(brokerId, + ZkUtils.readDataMaybeNull(zkClient, + ZkUtils.BrokerChannelsPath + "/" + brokerId + "/" + channelName)._1.get) } } def getExpectedChannelForBroker(zkClient: ZkClient, brokerId: Int, channelType: ChannelType): Option[ChannelInfo] = { -- 1.8.4.msysgit.0 From e4961f0674fc834642f97073a9139077c50f4c42 Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Wed, 22 Oct 2014 19:29:31 +0300 Subject: [PATCH 10/12] one more thing --- core/src/main/scala/kafka/server/KafkaHealthcheck.scala | 1 + core/src/main/scala/kafka/utils/ZkUtils.scala | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index b4fb050..000c98e 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -51,6 +51,7 @@ class KafkaHealthcheck(private val brokerId: Int, def shutdown() { zkClient.unsubscribeStateChanges(sessionExpireListener) ZkUtils.deregisterBrokerInZk(zkClient, brokerId) + ZkUtils.deregisterBrokerChannelsInZk(zkClient, brokerId) } /** diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 40eebb5..b1f8c41 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -211,6 +211,12 @@ object ZkUtils extends Logging { info("Deregistered broker %d at path %s.".format(id, brokerIdPath)) } + def deregisterBrokerChannelsInZk(zkClient: ZkClient, id: Int) { + val brokerChannelPath = ZkUtils.BrokerChannelsPath + "/" + id + deletePath(zkClient, brokerChannelPath) + info("Deregistered broker %d at path %s.".format(id, brokerChannelPath)) + } + def getConsumerPartitionOwnerPath(group: String, topic: String, partition: Int): String = { val topicDirs = new ZKGroupTopicDirs(group, topic) topicDirs.consumerOwnerDir + "/" + partition -- 1.8.4.msysgit.0 From 50f880776970f97adc294a5594f442847b19b753 Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Wed, 22 Oct 2014 19:36:24 +0300 Subject: [PATCH 11/12] one more thing --- config/server.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/config/server.properties b/config/server.properties index e22b27d..031cd28 100644 --- a/config/server.properties +++ b/config/server.properties @@ -22,7 +22,7 @@ broker.id=0 ############################# Socket Server Settings ############################# # The port the socket server listens on -#port=9092 +port=9092 secure.ssl.enabled=true -- 1.8.4.msysgit.0 From 614e9426918469b3b9cd5a0ca45ca6aafb66391a Mon Sep 17 00:00:00 2001 From: Ivan Lyutov Date: Thu, 23 Oct 2014 12:31:11 +0300 Subject: [PATCH 12/12] one more thing --- config/server.ssl.properties | 2 ++ docs/SECURITY.md | 44 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 46 insertions(+) create mode 100644 docs/SECURITY.md diff --git a/config/server.ssl.properties b/config/server.ssl.properties index 001424b..f088079 100644 --- a/config/server.ssl.properties +++ b/config/server.ssl.properties @@ -13,6 +13,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +port=9093 + #type of keystore keystore.type=jks diff --git a/docs/SECURITY.md b/docs/SECURITY.md new file mode 100644 index 0000000..3d459cd --- /dev/null +++ b/docs/SECURITY.md @@ -0,0 +1,44 @@ +SECURITY CONCEPT IMPLEMENTATION OVERVIEW +======================================== +# Motivation +Ability to support both secure and non-secure connection types on single kafka broker instance simultaneously and +provide simple approach for implementing new connection types without affecting existing clients. + +# Notion of channel +Channel is a high-level abstraction of connection type which could be simply represented as follows: + - port number (simply integer which represents the port to connect to). + - channel type (defines supported connection type and aware of the way how to initialize it). + +# Supported channel types +So far we support SSL and plaintext channels, but it easy to implement new one. + +# New config entries +secure.ssl.enable=(true|false) - defines whether SSL is enabled or disabled +Once it's enabled the configuration of SSL from file (server|client).ssl.properties is loaded to channel factory: +//Keystore file +keystore.type=jks +keystore=config/client.keystore +keystorePwd=test1234 +keyPwd=test1234 + +//Truststore file +truststore=config/client.keystore +truststorePwd=test1234 + +Server config has one extra field "port", which is 9093 by default. + +# Should we alter the broker model in order to contain one more port for SSL? +Since we don't want to affect existing clients the broker model should be as is. + +# Which port should be advertised in Zookeeper? +You are still free to decide which port to advertise in server config, but by default it is as follows: + 1) plaintext port (if it is enabled) + 2) SSL port (if it is enabled) + +# How should clients know about available channels then? +The information about channels is populated in Zookeeper at path "/brokers/channels/{brokerId}/{channelName}" on broker startup. +Thus, we are still backwards compatible. + +# Ok, how should producers know about channels? +The new request type was added MetadataForChannelRequest, which is the same MetadataRequest but has one extra field "channelType". +Thus, clients are able to tell broker about preferred connection type and clients that have no clue about security still can do it in old-fashion way. \ No newline at end of file -- 1.8.4.msysgit.0