mirror of
https://github.com/didi/KnowStreaming.git
synced 2026-01-06 05:22:16 +08:00
Add km module kafka
This commit is contained in:
18
tests/kafkatest/services/kafka/__init__.py
Normal file
18
tests/kafkatest/services/kafka/__init__.py
Normal file
@@ -0,0 +1,18 @@
|
||||
# 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.
|
||||
|
||||
from kafka import KafkaService
|
||||
from util import TopicPartition
|
||||
from config import KafkaConfig
|
||||
48
tests/kafkatest/services/kafka/config.py
Normal file
48
tests/kafkatest/services/kafka/config.py
Normal file
@@ -0,0 +1,48 @@
|
||||
# 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.
|
||||
|
||||
import config_property
|
||||
|
||||
|
||||
class KafkaConfig(dict):
|
||||
"""A dictionary-like container class which allows for definition of overridable default values,
|
||||
which is also capable of "rendering" itself as a useable server.properties file.
|
||||
"""
|
||||
|
||||
DEFAULTS = {
|
||||
config_property.PORT: 9092,
|
||||
config_property.SOCKET_RECEIVE_BUFFER_BYTES: 65536,
|
||||
config_property.LOG_DIRS: "/mnt/kafka/kafka-data-logs-1,/mnt/kafka/kafka-data-logs-2",
|
||||
config_property.ZOOKEEPER_CONNECTION_TIMEOUT_MS: 2000
|
||||
}
|
||||
|
||||
def __init__(self, **kwargs):
|
||||
super(KafkaConfig, self).__init__(**kwargs)
|
||||
|
||||
# Set defaults
|
||||
for key, val in self.DEFAULTS.items():
|
||||
if not self.has_key(key):
|
||||
self[key] = val
|
||||
|
||||
def render(self):
|
||||
"""Render self as a series of lines key=val\n, and do so in a consistent order. """
|
||||
keys = [k for k in self.keys()]
|
||||
keys.sort()
|
||||
|
||||
s = ""
|
||||
for k in keys:
|
||||
s += "%s=%s\n" % (k, str(self[k]))
|
||||
return s
|
||||
|
||||
192
tests/kafkatest/services/kafka/config_property.py
Normal file
192
tests/kafkatest/services/kafka/config_property.py
Normal file
@@ -0,0 +1,192 @@
|
||||
# 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.
|
||||
|
||||
"""
|
||||
Define Kafka configuration property names here.
|
||||
"""
|
||||
|
||||
BROKER_ID = "broker.id"
|
||||
PORT = "port"
|
||||
ADVERTISED_HOSTNAME = "advertised.host.name"
|
||||
|
||||
NUM_NETWORK_THREADS = "num.network.threads"
|
||||
NUM_IO_THREADS = "num.io.threads"
|
||||
SOCKET_SEND_BUFFER_BYTES = "socket.send.buffer.bytes"
|
||||
SOCKET_RECEIVE_BUFFER_BYTES = "socket.receive.buffer.bytes"
|
||||
SOCKET_REQUEST_MAX_BYTES = "socket.request.max.bytes"
|
||||
LOG_DIRS = "log.dirs"
|
||||
NUM_PARTITIONS = "num.partitions"
|
||||
NUM_RECOVERY_THREADS_PER_DATA_DIR = "num.recovery.threads.per.data.dir"
|
||||
|
||||
LOG_RETENTION_HOURS = "log.retention.hours"
|
||||
LOG_SEGMENT_BYTES = "log.segment.bytes"
|
||||
LOG_RETENTION_CHECK_INTERVAL_MS = "log.retention.check.interval.ms"
|
||||
LOG_RETENTION_MS = "log.retention.ms"
|
||||
LOG_CLEANER_ENABLE = "log.cleaner.enable"
|
||||
|
||||
AUTO_CREATE_TOPICS_ENABLE = "auto.create.topics.enable"
|
||||
|
||||
ZOOKEEPER_CONNECT = "zookeeper.connect"
|
||||
ZOOKEEPER_SSL_CLIENT_ENABLE = "zookeeper.ssl.client.enable"
|
||||
ZOOKEEPER_CLIENT_CNXN_SOCKET = "zookeeper.clientCnxnSocket"
|
||||
ZOOKEEPER_CONNECTION_TIMEOUT_MS = "zookeeper.connection.timeout.ms"
|
||||
INTER_BROKER_PROTOCOL_VERSION = "inter.broker.protocol.version"
|
||||
MESSAGE_FORMAT_VERSION = "log.message.format.version"
|
||||
MESSAGE_TIMESTAMP_TYPE = "message.timestamp.type"
|
||||
THROTTLING_REPLICATION_RATE_LIMIT = "replication.quota.throttled.rate"
|
||||
|
||||
LOG_FLUSH_INTERVAL_MESSAGE = "log.flush.interval.messages"
|
||||
REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS = "replica.high.watermark.checkpoint.interval.ms"
|
||||
LOG_ROLL_TIME_MS = "log.roll.ms"
|
||||
OFFSETS_TOPIC_NUM_PARTITIONS = "offsets.topic.num.partitions"
|
||||
|
||||
DELEGATION_TOKEN_MAX_LIFETIME_MS="delegation.token.max.lifetime.ms"
|
||||
DELEGATION_TOKEN_EXPIRY_TIME_MS="delegation.token.expiry.time.ms"
|
||||
DELEGATION_TOKEN_MASTER_KEY="delegation.token.master.key"
|
||||
SASL_ENABLED_MECHANISMS="sasl.enabled.mechanisms"
|
||||
|
||||
|
||||
"""
|
||||
From KafkaConfig.scala
|
||||
|
||||
/** ********* General Configuration ***********/
|
||||
val MaxReservedBrokerIdProp = "reserved.broker.max.id"
|
||||
val MessageMaxBytesProp = "message.max.bytes"
|
||||
val NumIoThreadsProp = "num.io.threads"
|
||||
val BackgroundThreadsProp = "background.threads"
|
||||
val QueuedMaxRequestsProp = "queued.max.requests"
|
||||
/** ********* Socket Server Configuration ***********/
|
||||
val PortProp = "port"
|
||||
val HostNameProp = "host.name"
|
||||
val ListenersProp = "listeners"
|
||||
val AdvertisedPortProp = "advertised.port"
|
||||
val AdvertisedListenersProp = "advertised.listeners"
|
||||
val SocketSendBufferBytesProp = "socket.send.buffer.bytes"
|
||||
val SocketReceiveBufferBytesProp = "socket.receive.buffer.bytes"
|
||||
val SocketRequestMaxBytesProp = "socket.request.max.bytes"
|
||||
val MaxConnectionsPerIpProp = "max.connections.per.ip"
|
||||
val MaxConnectionsPerIpOverridesProp = "max.connections.per.ip.overrides"
|
||||
val ConnectionsMaxIdleMsProp = "connections.max.idle.ms"
|
||||
/** ********* Log Configuration ***********/
|
||||
val NumPartitionsProp = "num.partitions"
|
||||
val LogDirsProp = "log.dirs"
|
||||
val LogDirProp = "log.dir"
|
||||
val LogSegmentBytesProp = "log.segment.bytes"
|
||||
|
||||
val LogRollTimeMillisProp = "log.roll.ms"
|
||||
val LogRollTimeHoursProp = "log.roll.hours"
|
||||
|
||||
val LogRollTimeJitterMillisProp = "log.roll.jitter.ms"
|
||||
val LogRollTimeJitterHoursProp = "log.roll.jitter.hours"
|
||||
|
||||
val LogRetentionTimeMillisProp = "log.retention.ms"
|
||||
val LogRetentionTimeMinutesProp = "log.retention.minutes"
|
||||
val LogRetentionTimeHoursProp = "log.retention.hours"
|
||||
|
||||
val LogRetentionBytesProp = "log.retention.bytes"
|
||||
val LogCleanupIntervalMsProp = "log.retention.check.interval.ms"
|
||||
val LogCleanupPolicyProp = "log.cleanup.policy"
|
||||
val LogCleanerThreadsProp = "log.cleaner.threads"
|
||||
val LogCleanerIoMaxBytesPerSecondProp = "log.cleaner.io.max.bytes.per.second"
|
||||
val LogCleanerDedupeBufferSizeProp = "log.cleaner.dedupe.buffer.size"
|
||||
val LogCleanerIoBufferSizeProp = "log.cleaner.io.buffer.size"
|
||||
val LogCleanerDedupeBufferLoadFactorProp = "log.cleaner.io.buffer.load.factor"
|
||||
val LogCleanerBackoffMsProp = "log.cleaner.backoff.ms"
|
||||
val LogCleanerMinCleanRatioProp = "log.cleaner.min.cleanable.ratio"
|
||||
val LogCleanerEnableProp = "log.cleaner.enable"
|
||||
val LogCleanerDeleteRetentionMsProp = "log.cleaner.delete.retention.ms"
|
||||
val LogIndexSizeMaxBytesProp = "log.index.size.max.bytes"
|
||||
val LogIndexIntervalBytesProp = "log.index.interval.bytes"
|
||||
val LogFlushIntervalMessagesProp = "log.flush.interval.messages"
|
||||
val LogDeleteDelayMsProp = "log.segment.delete.delay.ms"
|
||||
val LogFlushSchedulerIntervalMsProp = "log.flush.scheduler.interval.ms"
|
||||
val LogFlushIntervalMsProp = "log.flush.interval.ms"
|
||||
val LogFlushOffsetCheckpointIntervalMsProp = "log.flush.offset.checkpoint.interval.ms"
|
||||
val LogPreAllocateProp = "log.preallocate"
|
||||
val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir"
|
||||
val MinInSyncReplicasProp = "min.insync.replicas"
|
||||
/** ********* Replication configuration ***********/
|
||||
val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms"
|
||||
val DefaultReplicationFactorProp = "default.replication.factor"
|
||||
val ReplicaLagTimeMaxMsProp = "replica.lag.time.max.ms"
|
||||
val ReplicaSocketTimeoutMsProp = "replica.socket.timeout.ms"
|
||||
val ReplicaSocketReceiveBufferBytesProp = "replica.socket.receive.buffer.bytes"
|
||||
val ReplicaFetchMaxBytesProp = "replica.fetch.max.bytes"
|
||||
val ReplicaFetchWaitMaxMsProp = "replica.fetch.wait.max.ms"
|
||||
val ReplicaFetchMinBytesProp = "replica.fetch.min.bytes"
|
||||
val ReplicaFetchBackoffMsProp = "replica.fetch.backoff.ms"
|
||||
val NumReplicaFetchersProp = "num.replica.fetchers"
|
||||
val ReplicaHighWatermarkCheckpointIntervalMsProp = "replica.high.watermark.checkpoint.interval.ms"
|
||||
val FetchPurgatoryPurgeIntervalRequestsProp = "fetch.purgatory.purge.interval.requests"
|
||||
val ProducerPurgatoryPurgeIntervalRequestsProp = "producer.purgatory.purge.interval.requests"
|
||||
val AutoLeaderRebalanceEnableProp = "auto.leader.rebalance.enable"
|
||||
val LeaderImbalancePerBrokerPercentageProp = "leader.imbalance.per.broker.percentage"
|
||||
val LeaderImbalanceCheckIntervalSecondsProp = "leader.imbalance.check.interval.seconds"
|
||||
val UncleanLeaderElectionEnableProp = "unclean.leader.election.enable"
|
||||
val InterBrokerSecurityProtocolProp = "security.inter.broker.protocol"
|
||||
val InterBrokerProtocolVersionProp = "inter.broker.protocol.version"
|
||||
/** ********* Controlled shutdown configuration ***********/
|
||||
val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries"
|
||||
val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms"
|
||||
val ControlledShutdownEnableProp = "controlled.shutdown.enable"
|
||||
/** ********* Consumer coordinator configuration ***********/
|
||||
val ConsumerMinSessionTimeoutMsProp = "consumer.min.session.timeout.ms"
|
||||
val ConsumerMaxSessionTimeoutMsProp = "consumer.max.session.timeout.ms"
|
||||
/** ********* Offset management configuration ***********/
|
||||
val OffsetMetadataMaxSizeProp = "offset.metadata.max.bytes"
|
||||
val OffsetsLoadBufferSizeProp = "offsets.load.buffer.size"
|
||||
val OffsetsTopicReplicationFactorProp = "offsets.topic.replication.factor"
|
||||
val OffsetsTopicPartitionsProp = "offsets.topic.num.partitions"
|
||||
val OffsetsTopicSegmentBytesProp = "offsets.topic.segment.bytes"
|
||||
val OffsetsTopicCompressionCodecProp = "offsets.topic.compression.codec"
|
||||
val OffsetsRetentionMinutesProp = "offsets.retention.minutes"
|
||||
val OffsetsRetentionCheckIntervalMsProp = "offsets.retention.check.interval.ms"
|
||||
val OffsetCommitTimeoutMsProp = "offsets.commit.timeout.ms"
|
||||
val OffsetCommitRequiredAcksProp = "offsets.commit.required.acks"
|
||||
/** ********* Quota Configuration ***********/
|
||||
val ProducerQuotaBytesPerSecondDefaultProp = "quota.producer.default"
|
||||
val ConsumerQuotaBytesPerSecondDefaultProp = "quota.consumer.default"
|
||||
val NumQuotaSamplesProp = "quota.window.num"
|
||||
val QuotaWindowSizeSecondsProp = "quota.window.size.seconds"
|
||||
|
||||
val DeleteTopicEnableProp = "delete.topic.enable"
|
||||
val CompressionTypeProp = "compression.type"
|
||||
|
||||
/** ********* Kafka Metrics Configuration ***********/
|
||||
val MetricSampleWindowMsProp = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG
|
||||
val MetricNumSamplesProp: String = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG
|
||||
val MetricReporterClassesProp: String = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG
|
||||
|
||||
/** ********* SSL Configuration ****************/
|
||||
val PrincipalBuilderClassProp = SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG
|
||||
val SSLProtocolProp = SSLConfigs.SSL_PROTOCOL_CONFIG
|
||||
val SSLProviderProp = SSLConfigs.SSL_PROVIDER_CONFIG
|
||||
val SSLCipherSuitesProp = SSLConfigs.SSL_CIPHER_SUITES_CONFIG
|
||||
val SSLEnabledProtocolsProp = SSLConfigs.SSL_ENABLED_PROTOCOLS_CONFIG
|
||||
val SSLKeystoreTypeProp = SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG
|
||||
val SSLKeystoreLocationProp = SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG
|
||||
val SSLKeystorePasswordProp = SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG
|
||||
val SSLKeyPasswordProp = SSLConfigs.SSL_KEY_PASSWORD_CONFIG
|
||||
val SSLTruststoreTypeProp = SSLConfigs.SSL_TRUSTSTORE_TYPE_CONFIG
|
||||
val SSLTruststoreLocationProp = SSLConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG
|
||||
val SSLTruststorePasswordProp = SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG
|
||||
val SSLKeyManagerAlgorithmProp = SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG
|
||||
val SSLTrustManagerAlgorithmProp = SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG
|
||||
val SSLEndpointIdentificationAlgorithmProp = SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG
|
||||
val SSLSecureRandomImplementationProp = SSLConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG
|
||||
val SSLClientAuthProp = SSLConfigs.SSL_CLIENT_AUTH_CONFIG
|
||||
"""
|
||||
|
||||
|
||||
897
tests/kafkatest/services/kafka/kafka.py
Normal file
897
tests/kafkatest/services/kafka/kafka.py
Normal file
@@ -0,0 +1,897 @@
|
||||
# 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.
|
||||
|
||||
import collections
|
||||
import json
|
||||
import os.path
|
||||
import re
|
||||
import signal
|
||||
import time
|
||||
|
||||
from ducktape.services.service import Service
|
||||
from ducktape.utils.util import wait_until
|
||||
from ducktape.cluster.remoteaccount import RemoteCommandError
|
||||
|
||||
from config import KafkaConfig
|
||||
from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin
|
||||
from kafkatest.services.kafka import config_property
|
||||
from kafkatest.services.monitor.jmx import JmxMixin
|
||||
from kafkatest.services.security.minikdc import MiniKdc
|
||||
from kafkatest.services.security.listener_security_config import ListenerSecurityConfig
|
||||
from kafkatest.services.security.security_config import SecurityConfig
|
||||
from kafkatest.version import DEV_BRANCH, LATEST_0_10_0
|
||||
|
||||
|
||||
class KafkaListener:
|
||||
|
||||
def __init__(self, name, port_number, security_protocol, open=False):
|
||||
self.name = name
|
||||
self.port_number = port_number
|
||||
self.security_protocol = security_protocol
|
||||
self.open = open
|
||||
|
||||
def listener(self):
|
||||
return "%s://:%s" % (self.name, str(self.port_number))
|
||||
|
||||
def advertised_listener(self, node):
|
||||
return "%s://%s:%s" % (self.name, node.account.hostname, str(self.port_number))
|
||||
|
||||
def listener_security_protocol(self):
|
||||
return "%s:%s" % (self.name, self.security_protocol)
|
||||
|
||||
class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
||||
PERSISTENT_ROOT = "/mnt/kafka"
|
||||
STDOUT_STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "server-start-stdout-stderr.log")
|
||||
LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "kafka-log4j.properties")
|
||||
# Logs such as controller.log, server.log, etc all go here
|
||||
OPERATIONAL_LOG_DIR = os.path.join(PERSISTENT_ROOT, "kafka-operational-logs")
|
||||
OPERATIONAL_LOG_INFO_DIR = os.path.join(OPERATIONAL_LOG_DIR, "info")
|
||||
OPERATIONAL_LOG_DEBUG_DIR = os.path.join(OPERATIONAL_LOG_DIR, "debug")
|
||||
# Kafka log segments etc go here
|
||||
DATA_LOG_DIR_PREFIX = os.path.join(PERSISTENT_ROOT, "kafka-data-logs")
|
||||
DATA_LOG_DIR_1 = "%s-1" % (DATA_LOG_DIR_PREFIX)
|
||||
DATA_LOG_DIR_2 = "%s-2" % (DATA_LOG_DIR_PREFIX)
|
||||
CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "kafka.properties")
|
||||
# Kafka Authorizer
|
||||
ACL_AUTHORIZER = "kafka.security.authorizer.AclAuthorizer"
|
||||
# Old Kafka Authorizer. This is deprecated but still supported.
|
||||
SIMPLE_AUTHORIZER = "kafka.security.auth.SimpleAclAuthorizer"
|
||||
HEAP_DUMP_FILE = os.path.join(PERSISTENT_ROOT, "kafka_heap_dump.bin")
|
||||
INTERBROKER_LISTENER_NAME = 'INTERNAL'
|
||||
JAAS_CONF_PROPERTY = "java.security.auth.login.config=/mnt/security/jaas.conf"
|
||||
KRB5_CONF = "java.security.krb5.conf=/mnt/security/krb5.conf"
|
||||
|
||||
logs = {
|
||||
"kafka_server_start_stdout_stderr": {
|
||||
"path": STDOUT_STDERR_CAPTURE,
|
||||
"collect_default": True},
|
||||
"kafka_operational_logs_info": {
|
||||
"path": OPERATIONAL_LOG_INFO_DIR,
|
||||
"collect_default": True},
|
||||
"kafka_operational_logs_debug": {
|
||||
"path": OPERATIONAL_LOG_DEBUG_DIR,
|
||||
"collect_default": False},
|
||||
"kafka_data_1": {
|
||||
"path": DATA_LOG_DIR_1,
|
||||
"collect_default": False},
|
||||
"kafka_data_2": {
|
||||
"path": DATA_LOG_DIR_2,
|
||||
"collect_default": False},
|
||||
"kafka_heap_dump_file": {
|
||||
"path": HEAP_DUMP_FILE,
|
||||
"collect_default": True}
|
||||
}
|
||||
|
||||
def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAINTEXT, interbroker_security_protocol=SecurityConfig.PLAINTEXT,
|
||||
client_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI,
|
||||
authorizer_class_name=None, topics=None, version=DEV_BRANCH, jmx_object_names=None,
|
||||
jmx_attributes=None, zk_connect_timeout=5000, zk_session_timeout=6000, server_prop_overides=None, zk_chroot=None,
|
||||
zk_client_secure=False,
|
||||
listener_security_config=ListenerSecurityConfig(), per_node_server_prop_overrides=None, extra_kafka_opts=""):
|
||||
"""
|
||||
:param context: test context
|
||||
:param ZookeeperService zk:
|
||||
:param dict topics: which topics to create automatically
|
||||
:param str security_protocol: security protocol for clients to use
|
||||
:param str interbroker_security_protocol: security protocol to use for broker-to-broker communication
|
||||
:param str client_sasl_mechanism: sasl mechanism for clients to use
|
||||
:param str interbroker_sasl_mechanism: sasl mechanism to use for broker-to-broker communication
|
||||
:param str authorizer_class_name: which authorizer class to use
|
||||
:param str version: which kafka version to use. Defaults to "dev" branch
|
||||
:param jmx_object_names:
|
||||
:param jmx_attributes:
|
||||
:param int zk_connect_timeout:
|
||||
:param int zk_session_timeout:
|
||||
:param dict server_prop_overides: overrides for kafka.properties file
|
||||
:param zk_chroot:
|
||||
:param bool zk_client_secure: connect to Zookeeper over secure client port (TLS) when True
|
||||
:param ListenerSecurityConfig listener_security_config: listener config to use
|
||||
:param dict per_node_server_prop_overrides:
|
||||
:param str extra_kafka_opts: jvm args to add to KAFKA_OPTS variable
|
||||
"""
|
||||
Service.__init__(self, context, num_nodes)
|
||||
JmxMixin.__init__(self, num_nodes=num_nodes, jmx_object_names=jmx_object_names, jmx_attributes=(jmx_attributes or []),
|
||||
root=KafkaService.PERSISTENT_ROOT)
|
||||
|
||||
self.zk = zk
|
||||
|
||||
self.security_protocol = security_protocol
|
||||
self.client_sasl_mechanism = client_sasl_mechanism
|
||||
self.topics = topics
|
||||
self.minikdc = None
|
||||
self.authorizer_class_name = authorizer_class_name
|
||||
self.zk_set_acl = False
|
||||
if server_prop_overides is None:
|
||||
self.server_prop_overides = []
|
||||
else:
|
||||
self.server_prop_overides = server_prop_overides
|
||||
if per_node_server_prop_overrides is None:
|
||||
self.per_node_server_prop_overrides = {}
|
||||
else:
|
||||
self.per_node_server_prop_overrides = per_node_server_prop_overrides
|
||||
self.log_level = "DEBUG"
|
||||
self.zk_chroot = zk_chroot
|
||||
self.zk_client_secure = zk_client_secure
|
||||
self.listener_security_config = listener_security_config
|
||||
self.extra_kafka_opts = extra_kafka_opts
|
||||
|
||||
#
|
||||
# In a heavily loaded and not very fast machine, it is
|
||||
# sometimes necessary to give more time for the zk client
|
||||
# to have its session established, especially if the client
|
||||
# is authenticating and waiting for the SaslAuthenticated
|
||||
# in addition to the SyncConnected event.
|
||||
#
|
||||
# The default value for zookeeper.connect.timeout.ms is
|
||||
# 2 seconds and here we increase it to 5 seconds, but
|
||||
# it can be overridden by setting the corresponding parameter
|
||||
# for this constructor.
|
||||
self.zk_connect_timeout = zk_connect_timeout
|
||||
|
||||
# Also allow the session timeout to be provided explicitly,
|
||||
# primarily so that test cases can depend on it when waiting
|
||||
# e.g. brokers to deregister after a hard kill.
|
||||
self.zk_session_timeout = zk_session_timeout
|
||||
|
||||
self.port_mappings = {
|
||||
'PLAINTEXT': KafkaListener('PLAINTEXT', 9092, 'PLAINTEXT', False),
|
||||
'SSL': KafkaListener('SSL', 9093, 'SSL', False),
|
||||
'SASL_PLAINTEXT': KafkaListener('SASL_PLAINTEXT', 9094, 'SASL_PLAINTEXT', False),
|
||||
'SASL_SSL': KafkaListener('SASL_SSL', 9095, 'SASL_SSL', False),
|
||||
KafkaService.INTERBROKER_LISTENER_NAME:
|
||||
KafkaListener(KafkaService.INTERBROKER_LISTENER_NAME, 9099, None, False)
|
||||
}
|
||||
|
||||
self.interbroker_listener = None
|
||||
self.setup_interbroker_listener(interbroker_security_protocol, self.listener_security_config.use_separate_interbroker_listener)
|
||||
self.interbroker_sasl_mechanism = interbroker_sasl_mechanism
|
||||
|
||||
for node in self.nodes:
|
||||
node.version = version
|
||||
node.config = KafkaConfig(**{config_property.BROKER_ID: self.idx(node)})
|
||||
|
||||
def set_version(self, version):
|
||||
for node in self.nodes:
|
||||
node.version = version
|
||||
|
||||
@property
|
||||
def interbroker_security_protocol(self):
|
||||
return self.interbroker_listener.security_protocol
|
||||
|
||||
# this is required for backwards compatibility - there are a lot of tests that set this property explicitly
|
||||
# meaning 'use one of the existing listeners that match given security protocol, do not use custom listener'
|
||||
@interbroker_security_protocol.setter
|
||||
def interbroker_security_protocol(self, security_protocol):
|
||||
self.setup_interbroker_listener(security_protocol, use_separate_listener=False)
|
||||
|
||||
def setup_interbroker_listener(self, security_protocol, use_separate_listener=False):
|
||||
self.listener_security_config.use_separate_interbroker_listener = use_separate_listener
|
||||
|
||||
if self.listener_security_config.use_separate_interbroker_listener:
|
||||
# do not close existing port here since it is not used exclusively for interbroker communication
|
||||
self.interbroker_listener = self.port_mappings[KafkaService.INTERBROKER_LISTENER_NAME]
|
||||
self.interbroker_listener.security_protocol = security_protocol
|
||||
else:
|
||||
# close dedicated interbroker port, so it's not dangling in 'listeners' and 'advertised.listeners'
|
||||
self.close_port(KafkaService.INTERBROKER_LISTENER_NAME)
|
||||
self.interbroker_listener = self.port_mappings[security_protocol]
|
||||
|
||||
@property
|
||||
def security_config(self):
|
||||
config = SecurityConfig(self.context, self.security_protocol, self.interbroker_listener.security_protocol,
|
||||
zk_sasl=self.zk.zk_sasl, zk_tls=self.zk_client_secure,
|
||||
client_sasl_mechanism=self.client_sasl_mechanism,
|
||||
interbroker_sasl_mechanism=self.interbroker_sasl_mechanism,
|
||||
listener_security_config=self.listener_security_config)
|
||||
for port in self.port_mappings.values():
|
||||
if port.open:
|
||||
config.enable_security_protocol(port.security_protocol)
|
||||
return config
|
||||
|
||||
def open_port(self, listener_name):
|
||||
self.port_mappings[listener_name].open = True
|
||||
|
||||
def close_port(self, listener_name):
|
||||
self.port_mappings[listener_name].open = False
|
||||
|
||||
def start_minikdc_if_necessary(self, add_principals=""):
|
||||
if self.security_config.has_sasl:
|
||||
if self.minikdc is None:
|
||||
self.minikdc = MiniKdc(self.context, self.nodes, extra_principals = add_principals)
|
||||
self.minikdc.start()
|
||||
else:
|
||||
self.minikdc = None
|
||||
|
||||
def alive(self, node):
|
||||
return len(self.pids(node)) > 0
|
||||
|
||||
def start(self, add_principals="", use_zk_to_create_topic=True):
|
||||
if self.zk_client_secure and not self.zk.zk_client_secure_port:
|
||||
raise Exception("Unable to start Kafka: TLS to Zookeeper requested but Zookeeper secure port not enabled")
|
||||
self.open_port(self.security_protocol)
|
||||
self.interbroker_listener.open = True
|
||||
|
||||
self.start_minikdc_if_necessary(add_principals)
|
||||
self._ensure_zk_chroot()
|
||||
|
||||
Service.start(self)
|
||||
|
||||
self.logger.info("Waiting for brokers to register at ZK")
|
||||
|
||||
retries = 30
|
||||
expected_broker_ids = set(self.nodes)
|
||||
wait_until(lambda: {node for node in self.nodes if self.is_registered(node)} == expected_broker_ids, 30, 1)
|
||||
|
||||
if retries == 0:
|
||||
raise RuntimeError("Kafka servers didn't register at ZK within 30 seconds")
|
||||
|
||||
# Create topics if necessary
|
||||
if self.topics is not None:
|
||||
for topic, topic_cfg in self.topics.items():
|
||||
if topic_cfg is None:
|
||||
topic_cfg = {}
|
||||
|
||||
topic_cfg["topic"] = topic
|
||||
self.create_topic(topic_cfg, use_zk_to_create_topic=use_zk_to_create_topic)
|
||||
|
||||
def _ensure_zk_chroot(self):
|
||||
self.logger.info("Ensuring zk_chroot %s exists", self.zk_chroot)
|
||||
if self.zk_chroot:
|
||||
if not self.zk_chroot.startswith('/'):
|
||||
raise Exception("Zookeeper chroot must start with '/' but found " + self.zk_chroot)
|
||||
|
||||
parts = self.zk_chroot.split('/')[1:]
|
||||
for i in range(len(parts)):
|
||||
self.zk.create('/' + '/'.join(parts[:i+1]))
|
||||
|
||||
def set_protocol_and_port(self, node):
|
||||
listeners = []
|
||||
advertised_listeners = []
|
||||
protocol_map = []
|
||||
|
||||
for port in self.port_mappings.values():
|
||||
if port.open:
|
||||
listeners.append(port.listener())
|
||||
advertised_listeners.append(port.advertised_listener(node))
|
||||
protocol_map.append(port.listener_security_protocol())
|
||||
|
||||
self.listeners = ','.join(listeners)
|
||||
self.advertised_listeners = ','.join(advertised_listeners)
|
||||
self.listener_security_protocol_map = ','.join(protocol_map)
|
||||
self.interbroker_bootstrap_servers = self.__bootstrap_servers(self.interbroker_listener, True)
|
||||
|
||||
def prop_file(self, node):
|
||||
self.set_protocol_and_port(node)
|
||||
|
||||
#load template configs as dictionary
|
||||
config_template = self.render('kafka.properties', node=node, broker_id=self.idx(node),
|
||||
security_config=self.security_config, num_nodes=self.num_nodes,
|
||||
listener_security_config=self.listener_security_config)
|
||||
|
||||
configs = dict( l.rstrip().split('=', 1) for l in config_template.split('\n')
|
||||
if not l.startswith("#") and "=" in l )
|
||||
|
||||
#load specific test override configs
|
||||
override_configs = KafkaConfig(**node.config)
|
||||
override_configs[config_property.ADVERTISED_HOSTNAME] = node.account.hostname
|
||||
override_configs[config_property.ZOOKEEPER_CONNECT] = self.zk_connect_setting()
|
||||
if self.zk_client_secure:
|
||||
override_configs[config_property.ZOOKEEPER_SSL_CLIENT_ENABLE] = 'true'
|
||||
override_configs[config_property.ZOOKEEPER_CLIENT_CNXN_SOCKET] = 'org.apache.zookeeper.ClientCnxnSocketNetty'
|
||||
else:
|
||||
override_configs[config_property.ZOOKEEPER_SSL_CLIENT_ENABLE] = 'false'
|
||||
|
||||
for prop in self.server_prop_overides:
|
||||
override_configs[prop[0]] = prop[1]
|
||||
|
||||
for prop in self.per_node_server_prop_overrides.get(self.idx(node), []):
|
||||
override_configs[prop[0]] = prop[1]
|
||||
|
||||
#update template configs with test override configs
|
||||
configs.update(override_configs)
|
||||
|
||||
prop_file = self.render_configs(configs)
|
||||
return prop_file
|
||||
|
||||
def render_configs(self, configs):
|
||||
"""Render self as a series of lines key=val\n, and do so in a consistent order. """
|
||||
keys = [k for k in configs.keys()]
|
||||
keys.sort()
|
||||
|
||||
s = ""
|
||||
for k in keys:
|
||||
s += "%s=%s\n" % (k, str(configs[k]))
|
||||
return s
|
||||
|
||||
def start_cmd(self, node):
|
||||
cmd = "export JMX_PORT=%d; " % self.jmx_port
|
||||
cmd += "export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % self.LOG4J_CONFIG
|
||||
heap_kafka_opts = "-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=%s" % \
|
||||
self.logs["kafka_heap_dump_file"]["path"]
|
||||
security_kafka_opts = self.security_config.kafka_opts.strip('\"')
|
||||
cmd += "export KAFKA_OPTS=\"%s %s %s\"; " % (heap_kafka_opts, security_kafka_opts, self.extra_kafka_opts)
|
||||
cmd += "%s %s 1>> %s 2>> %s &" % \
|
||||
(self.path.script("kafka-server-start.sh", node),
|
||||
KafkaService.CONFIG_FILE,
|
||||
KafkaService.STDOUT_STDERR_CAPTURE,
|
||||
KafkaService.STDOUT_STDERR_CAPTURE)
|
||||
return cmd
|
||||
|
||||
def start_node(self, node, timeout_sec=60):
|
||||
node.account.mkdirs(KafkaService.PERSISTENT_ROOT)
|
||||
prop_file = self.prop_file(node)
|
||||
self.logger.info("kafka.properties:")
|
||||
self.logger.info(prop_file)
|
||||
node.account.create_file(KafkaService.CONFIG_FILE, prop_file)
|
||||
node.account.create_file(self.LOG4J_CONFIG, self.render('log4j.properties', log_dir=KafkaService.OPERATIONAL_LOG_DIR))
|
||||
|
||||
self.security_config.setup_node(node)
|
||||
self.security_config.setup_credentials(node, self.path, self.zk_connect_setting(), broker=True)
|
||||
|
||||
cmd = self.start_cmd(node)
|
||||
self.logger.debug("Attempting to start KafkaService on %s with command: %s" % (str(node.account), cmd))
|
||||
with node.account.monitor_log(KafkaService.STDOUT_STDERR_CAPTURE) as monitor:
|
||||
node.account.ssh(cmd)
|
||||
# Kafka 1.0.0 and higher don't have a space between "Kafka" and "Server"
|
||||
monitor.wait_until("Kafka\s*Server.*started", timeout_sec=timeout_sec, backoff_sec=.25,
|
||||
err_msg="Kafka server didn't finish startup in %d seconds" % timeout_sec)
|
||||
|
||||
# Credentials for inter-broker communication are created before starting Kafka.
|
||||
# Client credentials are created after starting Kafka so that both loading of
|
||||
# existing credentials from ZK and dynamic update of credentials in Kafka are tested.
|
||||
self.security_config.setup_credentials(node, self.path, self.zk_connect_setting(), broker=False)
|
||||
|
||||
self.start_jmx_tool(self.idx(node), node)
|
||||
if len(self.pids(node)) == 0:
|
||||
raise Exception("No process ids recorded on node %s" % node.account.hostname)
|
||||
|
||||
def pids(self, node):
|
||||
"""Return process ids associated with running processes on the given node."""
|
||||
try:
|
||||
cmd = "jcmd | grep -e %s | awk '{print $1}'" % self.java_class_name()
|
||||
pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)]
|
||||
return pid_arr
|
||||
except (RemoteCommandError, ValueError) as e:
|
||||
return []
|
||||
|
||||
def signal_node(self, node, sig=signal.SIGTERM):
|
||||
pids = self.pids(node)
|
||||
for pid in pids:
|
||||
node.account.signal(pid, sig)
|
||||
|
||||
def signal_leader(self, topic, partition=0, sig=signal.SIGTERM):
|
||||
leader = self.leader(topic, partition)
|
||||
self.signal_node(leader, sig)
|
||||
|
||||
def stop_node(self, node, clean_shutdown=True, timeout_sec=60):
|
||||
pids = self.pids(node)
|
||||
sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL
|
||||
|
||||
for pid in pids:
|
||||
node.account.signal(pid, sig, allow_fail=False)
|
||||
|
||||
try:
|
||||
wait_until(lambda: len(self.pids(node)) == 0, timeout_sec=timeout_sec,
|
||||
err_msg="Kafka node failed to stop in %d seconds" % timeout_sec)
|
||||
except Exception:
|
||||
self.thread_dump(node)
|
||||
raise
|
||||
|
||||
def thread_dump(self, node):
|
||||
for pid in self.pids(node):
|
||||
try:
|
||||
node.account.signal(pid, signal.SIGQUIT, allow_fail=True)
|
||||
except:
|
||||
self.logger.warn("Could not dump threads on node")
|
||||
|
||||
def clean_node(self, node):
|
||||
JmxMixin.clean_node(self, node)
|
||||
self.security_config.clean_node(node)
|
||||
node.account.kill_java_processes(self.java_class_name(),
|
||||
clean_shutdown=False, allow_fail=True)
|
||||
node.account.ssh("sudo rm -rf -- %s" % KafkaService.PERSISTENT_ROOT, allow_fail=False)
|
||||
|
||||
def _kafka_topics_cmd(self, node, use_zk_connection=True):
|
||||
"""
|
||||
Returns kafka-topics.sh command path with jaas configuration and krb5 environment variable
|
||||
set. If Admin client is not going to be used, don't set the environment variable.
|
||||
"""
|
||||
kafka_topic_script = self.path.script("kafka-topics.sh", node)
|
||||
skip_security_settings = use_zk_connection or not node.version.topic_command_supports_bootstrap_server()
|
||||
return kafka_topic_script if skip_security_settings else \
|
||||
"KAFKA_OPTS='-D%s -D%s' %s" % (KafkaService.JAAS_CONF_PROPERTY, KafkaService.KRB5_CONF, kafka_topic_script)
|
||||
|
||||
def _kafka_topics_cmd_config(self, node, use_zk_connection=True):
|
||||
"""
|
||||
Return --command-config parameter to the kafka-topics.sh command. The config parameter specifies
|
||||
the security settings that AdminClient uses to connect to a secure kafka server.
|
||||
"""
|
||||
skip_command_config = use_zk_connection or not node.version.topic_command_supports_bootstrap_server()
|
||||
return "" if skip_command_config else " --command-config <(echo '%s')" % (self.security_config.client_config())
|
||||
|
||||
def create_topic(self, topic_cfg, node=None, use_zk_to_create_topic=True):
|
||||
"""Run the admin tool create topic command.
|
||||
Specifying node is optional, and may be done if for different kafka nodes have different versions,
|
||||
and we care where command gets run.
|
||||
|
||||
If the node is not specified, run the command from self.nodes[0]
|
||||
"""
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
self.logger.info("Creating topic %s with settings %s",
|
||||
topic_cfg["topic"], topic_cfg)
|
||||
|
||||
use_zk_connection = topic_cfg.get('if-not-exists', False) or use_zk_to_create_topic
|
||||
|
||||
cmd = "%(kafka_topics_cmd)s %(connection_string)s --create --topic %(topic)s " % {
|
||||
'kafka_topics_cmd': self._kafka_topics_cmd(node, use_zk_connection),
|
||||
'connection_string': self._connect_setting(node, use_zk_connection),
|
||||
'topic': topic_cfg.get("topic"),
|
||||
}
|
||||
if 'replica-assignment' in topic_cfg:
|
||||
cmd += " --replica-assignment %(replica-assignment)s" % {
|
||||
'replica-assignment': topic_cfg.get('replica-assignment')
|
||||
}
|
||||
else:
|
||||
cmd += " --partitions %(partitions)d --replication-factor %(replication-factor)d" % {
|
||||
'partitions': topic_cfg.get('partitions', 1),
|
||||
'replication-factor': topic_cfg.get('replication-factor', 1)
|
||||
}
|
||||
|
||||
if topic_cfg.get('if-not-exists', False):
|
||||
cmd += ' --if-not-exists'
|
||||
|
||||
if "configs" in topic_cfg.keys() and topic_cfg["configs"] is not None:
|
||||
for config_name, config_value in topic_cfg["configs"].items():
|
||||
cmd += " --config %s=%s" % (config_name, str(config_value))
|
||||
|
||||
cmd += self._kafka_topics_cmd_config(node, use_zk_connection)
|
||||
|
||||
self.logger.info("Running topic creation command...\n%s" % cmd)
|
||||
node.account.ssh(cmd)
|
||||
|
||||
def delete_topic(self, topic, node=None):
|
||||
"""
|
||||
Delete a topic with the topics command
|
||||
:param topic:
|
||||
:param node:
|
||||
:return:
|
||||
"""
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
self.logger.info("Deleting topic %s" % topic)
|
||||
kafka_topic_script = self.path.script("kafka-topics.sh", node)
|
||||
|
||||
cmd = kafka_topic_script + " "
|
||||
cmd += "--bootstrap-server %(bootstrap_servers)s --delete --topic %(topic)s " % {
|
||||
'bootstrap_servers': self.bootstrap_servers(self.security_protocol),
|
||||
'topic': topic
|
||||
}
|
||||
self.logger.info("Running topic delete command...\n%s" % cmd)
|
||||
node.account.ssh(cmd)
|
||||
|
||||
def describe_topic(self, topic, node=None, use_zk_to_describe_topic=True):
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
cmd = "%s %s --topic %s --describe %s" % \
|
||||
(self._kafka_topics_cmd(node=node, use_zk_connection=use_zk_to_describe_topic),
|
||||
self._connect_setting(node=node, use_zk_connection=use_zk_to_describe_topic),
|
||||
topic, self._kafka_topics_cmd_config(node=node, use_zk_connection=use_zk_to_describe_topic))
|
||||
|
||||
self.logger.info("Running topic describe command...\n%s" % cmd)
|
||||
output = ""
|
||||
for line in node.account.ssh_capture(cmd):
|
||||
output += line
|
||||
return output
|
||||
|
||||
def list_topics(self, node=None, use_zk_to_list_topic=True):
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
cmd = "%s %s --list %s" % (self._kafka_topics_cmd(node, use_zk_to_list_topic),
|
||||
self._connect_setting(node, use_zk_to_list_topic),
|
||||
self._kafka_topics_cmd_config(node, use_zk_to_list_topic))
|
||||
for line in node.account.ssh_capture(cmd):
|
||||
if not line.startswith("SLF4J"):
|
||||
yield line.rstrip()
|
||||
|
||||
def alter_message_format(self, topic, msg_format_version, node=None):
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
self.logger.info("Altering message format version for topic %s with format %s", topic, msg_format_version)
|
||||
cmd = "%s --zookeeper %s %s --entity-name %s --entity-type topics --alter --add-config message.format.version=%s" % \
|
||||
(self.path.script("kafka-configs.sh", node), self.zk_connect_setting(), self.zk.zkTlsConfigFileOption(), topic, msg_format_version)
|
||||
self.logger.info("Running alter message format command...\n%s" % cmd)
|
||||
node.account.ssh(cmd)
|
||||
|
||||
def set_unclean_leader_election(self, topic, value=True, node=None):
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
if value is True:
|
||||
self.logger.info("Enabling unclean leader election for topic %s", topic)
|
||||
else:
|
||||
self.logger.info("Disabling unclean leader election for topic %s", topic)
|
||||
cmd = "%s --zookeeper %s %s --entity-name %s --entity-type topics --alter --add-config unclean.leader.election.enable=%s" % \
|
||||
(self.path.script("kafka-configs.sh", node), self.zk_connect_setting(), self.zk.zkTlsConfigFileOption(), topic, str(value).lower())
|
||||
self.logger.info("Running alter unclean leader command...\n%s" % cmd)
|
||||
node.account.ssh(cmd)
|
||||
|
||||
def parse_describe_topic(self, topic_description):
|
||||
"""Parse output of kafka-topics.sh --describe (or describe_topic() method above), which is a string of form
|
||||
PartitionCount:2\tReplicationFactor:2\tConfigs:
|
||||
Topic: test_topic\ttPartition: 0\tLeader: 3\tReplicas: 3,1\tIsr: 3,1
|
||||
Topic: test_topic\tPartition: 1\tLeader: 1\tReplicas: 1,2\tIsr: 1,2
|
||||
into a dictionary structure appropriate for use with reassign-partitions tool:
|
||||
{
|
||||
"partitions": [
|
||||
{"topic": "test_topic", "partition": 0, "replicas": [3, 1]},
|
||||
{"topic": "test_topic", "partition": 1, "replicas": [1, 2]}
|
||||
]
|
||||
}
|
||||
"""
|
||||
lines = map(lambda x: x.strip(), topic_description.split("\n"))
|
||||
partitions = []
|
||||
for line in lines:
|
||||
m = re.match(".*Leader:.*", line)
|
||||
if m is None:
|
||||
continue
|
||||
|
||||
fields = line.split("\t")
|
||||
# ["Partition: 4", "Leader: 0"] -> ["4", "0"]
|
||||
fields = map(lambda x: x.split(" ")[1], fields)
|
||||
partitions.append(
|
||||
{"topic": fields[0],
|
||||
"partition": int(fields[1]),
|
||||
"replicas": map(int, fields[3].split(','))})
|
||||
return {"partitions": partitions}
|
||||
|
||||
def verify_reassign_partitions(self, reassignment, node=None):
|
||||
"""Run the reassign partitions admin tool in "verify" mode
|
||||
"""
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
|
||||
json_file = "/tmp/%s_reassign.json" % str(time.time())
|
||||
|
||||
# reassignment to json
|
||||
json_str = json.dumps(reassignment)
|
||||
json_str = json.dumps(json_str)
|
||||
|
||||
# create command
|
||||
cmd = "echo %s > %s && " % (json_str, json_file)
|
||||
cmd += "%s " % self.path.script("kafka-reassign-partitions.sh", node)
|
||||
cmd += "--zookeeper %s " % self.zk_connect_setting()
|
||||
cmd += "--reassignment-json-file %s " % json_file
|
||||
cmd += "--verify "
|
||||
cmd += "&& sleep 1 && rm -f %s" % json_file
|
||||
|
||||
# send command
|
||||
self.logger.info("Verifying partition reassignment...")
|
||||
self.logger.debug(cmd)
|
||||
output = ""
|
||||
for line in node.account.ssh_capture(cmd):
|
||||
output += line
|
||||
|
||||
self.logger.debug(output)
|
||||
|
||||
if re.match(".*Reassignment of partition.*failed.*",
|
||||
output.replace('\n', '')) is not None:
|
||||
return False
|
||||
|
||||
if re.match(".*is still in progress.*",
|
||||
output.replace('\n', '')) is not None:
|
||||
return False
|
||||
|
||||
return True
|
||||
|
||||
def execute_reassign_partitions(self, reassignment, node=None,
|
||||
throttle=None):
|
||||
"""Run the reassign partitions admin tool in "verify" mode
|
||||
"""
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
json_file = "/tmp/%s_reassign.json" % str(time.time())
|
||||
|
||||
# reassignment to json
|
||||
json_str = json.dumps(reassignment)
|
||||
json_str = json.dumps(json_str)
|
||||
|
||||
# create command
|
||||
cmd = "echo %s > %s && " % (json_str, json_file)
|
||||
cmd += "%s " % self.path.script( "kafka-reassign-partitions.sh", node)
|
||||
cmd += "--zookeeper %s " % self.zk_connect_setting()
|
||||
cmd += "--reassignment-json-file %s " % json_file
|
||||
cmd += "--execute"
|
||||
if throttle is not None:
|
||||
cmd += " --throttle %d" % throttle
|
||||
cmd += " && sleep 1 && rm -f %s" % json_file
|
||||
|
||||
# send command
|
||||
self.logger.info("Executing parition reassignment...")
|
||||
self.logger.debug(cmd)
|
||||
output = ""
|
||||
for line in node.account.ssh_capture(cmd):
|
||||
output += line
|
||||
|
||||
self.logger.debug("Verify partition reassignment:")
|
||||
self.logger.debug(output)
|
||||
|
||||
def search_data_files(self, topic, messages):
|
||||
"""Check if a set of messages made it into the Kakfa data files. Note that
|
||||
this method takes no account of replication. It simply looks for the
|
||||
payload in all the partition files of the specified topic. 'messages' should be
|
||||
an array of numbers. The list of missing messages is returned.
|
||||
"""
|
||||
payload_match = "payload: " + "$|payload: ".join(str(x) for x in messages) + "$"
|
||||
found = set([])
|
||||
self.logger.debug("number of unique missing messages we will search for: %d",
|
||||
len(messages))
|
||||
for node in self.nodes:
|
||||
# Grab all .log files in directories prefixed with this topic
|
||||
files = node.account.ssh_capture("find %s* -regex '.*/%s-.*/[^/]*.log'" % (KafkaService.DATA_LOG_DIR_PREFIX, topic))
|
||||
|
||||
# Check each data file to see if it contains the messages we want
|
||||
for log in files:
|
||||
cmd = "%s kafka.tools.DumpLogSegments --print-data-log --files %s | grep -E \"%s\"" % \
|
||||
(self.path.script("kafka-run-class.sh", node), log.strip(), payload_match)
|
||||
|
||||
for line in node.account.ssh_capture(cmd, allow_fail=True):
|
||||
for val in messages:
|
||||
if line.strip().endswith("payload: "+str(val)):
|
||||
self.logger.debug("Found %s in data-file [%s] in line: [%s]" % (val, log.strip(), line.strip()))
|
||||
found.add(val)
|
||||
|
||||
self.logger.debug("Number of unique messages found in the log: %d",
|
||||
len(found))
|
||||
missing = list(set(messages) - found)
|
||||
|
||||
if len(missing) > 0:
|
||||
self.logger.warn("The following values were not found in the data files: " + str(missing))
|
||||
|
||||
return missing
|
||||
|
||||
def restart_cluster(self, clean_shutdown=True, timeout_sec=60, after_each_broker_restart=None, *args):
|
||||
for node in self.nodes:
|
||||
self.restart_node(node, clean_shutdown=clean_shutdown, timeout_sec=timeout_sec)
|
||||
if after_each_broker_restart is not None:
|
||||
after_each_broker_restart(*args)
|
||||
|
||||
def restart_node(self, node, clean_shutdown=True, timeout_sec=60):
|
||||
"""Restart the given node."""
|
||||
self.stop_node(node, clean_shutdown, timeout_sec)
|
||||
self.start_node(node, timeout_sec)
|
||||
|
||||
def isr_idx_list(self, topic, partition=0):
|
||||
""" Get in-sync replica list the given topic and partition.
|
||||
"""
|
||||
self.logger.debug("Querying zookeeper to find in-sync replicas for topic %s and partition %d" % (topic, partition))
|
||||
zk_path = "/brokers/topics/%s/partitions/%d/state" % (topic, partition)
|
||||
partition_state = self.zk.query(zk_path, chroot=self.zk_chroot)
|
||||
|
||||
if partition_state is None:
|
||||
raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition))
|
||||
|
||||
partition_state = json.loads(partition_state)
|
||||
self.logger.info(partition_state)
|
||||
|
||||
isr_idx_list = partition_state["isr"]
|
||||
self.logger.info("Isr for topic %s and partition %d is now: %s" % (topic, partition, isr_idx_list))
|
||||
return isr_idx_list
|
||||
|
||||
def replicas(self, topic, partition=0):
|
||||
""" Get the assigned replicas for the given topic and partition.
|
||||
"""
|
||||
self.logger.debug("Querying zookeeper to find assigned replicas for topic %s and partition %d" % (topic, partition))
|
||||
zk_path = "/brokers/topics/%s" % (topic)
|
||||
assignment = self.zk.query(zk_path, chroot=self.zk_chroot)
|
||||
|
||||
if assignment is None:
|
||||
raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition))
|
||||
|
||||
assignment = json.loads(assignment)
|
||||
self.logger.info(assignment)
|
||||
|
||||
replicas = assignment["partitions"][str(partition)]
|
||||
|
||||
self.logger.info("Assigned replicas for topic %s and partition %d is now: %s" % (topic, partition, replicas))
|
||||
return [self.get_node(replica) for replica in replicas]
|
||||
|
||||
def leader(self, topic, partition=0):
|
||||
""" Get the leader replica for the given topic and partition.
|
||||
"""
|
||||
self.logger.debug("Querying zookeeper to find leader replica for topic %s and partition %d" % (topic, partition))
|
||||
zk_path = "/brokers/topics/%s/partitions/%d/state" % (topic, partition)
|
||||
partition_state = self.zk.query(zk_path, chroot=self.zk_chroot)
|
||||
|
||||
if partition_state is None:
|
||||
raise Exception("Error finding partition state for topic %s and partition %d." % (topic, partition))
|
||||
|
||||
partition_state = json.loads(partition_state)
|
||||
self.logger.info(partition_state)
|
||||
|
||||
leader_idx = int(partition_state["leader"])
|
||||
self.logger.info("Leader for topic %s and partition %d is now: %d" % (topic, partition, leader_idx))
|
||||
return self.get_node(leader_idx)
|
||||
|
||||
def cluster_id(self):
|
||||
""" Get the current cluster id
|
||||
"""
|
||||
self.logger.debug("Querying ZooKeeper to retrieve cluster id")
|
||||
cluster = self.zk.query("/cluster/id", chroot=self.zk_chroot)
|
||||
|
||||
try:
|
||||
return json.loads(cluster)['id'] if cluster else None
|
||||
except:
|
||||
self.logger.debug("Data in /cluster/id znode could not be parsed. Data = %s" % cluster)
|
||||
raise
|
||||
|
||||
def check_protocol_errors(self, node):
|
||||
""" Checks for common protocol exceptions due to invalid inter broker protocol handling.
|
||||
While such errors can and should be checked in other ways, checking the logs is a worthwhile failsafe.
|
||||
"""
|
||||
for node in self.nodes:
|
||||
exit_code = node.account.ssh("grep -e 'java.lang.IllegalArgumentException: Invalid version' -e SchemaException %s/*"
|
||||
% KafkaService.OPERATIONAL_LOG_DEBUG_DIR, allow_fail=True)
|
||||
if exit_code != 1:
|
||||
return False
|
||||
return True
|
||||
|
||||
def list_consumer_groups(self, node=None, command_config=None):
|
||||
""" Get list of consumer groups.
|
||||
"""
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
consumer_group_script = self.path.script("kafka-consumer-groups.sh", node)
|
||||
|
||||
if command_config is None:
|
||||
command_config = ""
|
||||
else:
|
||||
command_config = "--command-config " + command_config
|
||||
|
||||
cmd = "%s --bootstrap-server %s %s --list" % \
|
||||
(consumer_group_script,
|
||||
self.bootstrap_servers(self.security_protocol),
|
||||
command_config)
|
||||
output = ""
|
||||
self.logger.debug(cmd)
|
||||
for line in node.account.ssh_capture(cmd):
|
||||
if not line.startswith("SLF4J"):
|
||||
output += line
|
||||
self.logger.debug(output)
|
||||
return output
|
||||
|
||||
def describe_consumer_group(self, group, node=None, command_config=None):
|
||||
""" Describe a consumer group.
|
||||
"""
|
||||
if node is None:
|
||||
node = self.nodes[0]
|
||||
consumer_group_script = self.path.script("kafka-consumer-groups.sh", node)
|
||||
|
||||
if command_config is None:
|
||||
command_config = ""
|
||||
else:
|
||||
command_config = "--command-config " + command_config
|
||||
|
||||
cmd = "%s --bootstrap-server %s %s --group %s --describe" % \
|
||||
(consumer_group_script,
|
||||
self.bootstrap_servers(self.security_protocol),
|
||||
command_config, group)
|
||||
|
||||
output = ""
|
||||
self.logger.debug(cmd)
|
||||
for line in node.account.ssh_capture(cmd):
|
||||
if not (line.startswith("SLF4J") or line.startswith("TOPIC") or line.startswith("Could not fetch offset")):
|
||||
output += line
|
||||
self.logger.debug(output)
|
||||
return output
|
||||
|
||||
def zk_connect_setting(self):
|
||||
return self.zk.connect_setting(self.zk_chroot, self.zk_client_secure)
|
||||
|
||||
def _connect_setting(self, node, use_zk_connection=True):
|
||||
"""
|
||||
Checks if --bootstrap-server config is supported, if yes then returns a string with
|
||||
bootstrap server, otherwise returns zookeeper connection string.
|
||||
"""
|
||||
if node.version.topic_command_supports_bootstrap_server() and not use_zk_connection:
|
||||
connection_setting = "--bootstrap-server %s" % (self.bootstrap_servers(self.security_protocol))
|
||||
else:
|
||||
connection_setting = "--zookeeper %s" % (self.zk_connect_setting())
|
||||
|
||||
return connection_setting
|
||||
|
||||
def __bootstrap_servers(self, port, validate=True, offline_nodes=[]):
|
||||
if validate and not port.open:
|
||||
raise ValueError("We are retrieving bootstrap servers for the port: %s which is not currently open. - " %
|
||||
str(port.port_number))
|
||||
|
||||
return ','.join([node.account.hostname + ":" + str(port.port_number)
|
||||
for node in self.nodes
|
||||
if node not in offline_nodes])
|
||||
|
||||
def bootstrap_servers(self, protocol='PLAINTEXT', validate=True, offline_nodes=[]):
|
||||
"""Return comma-delimited list of brokers in this cluster formatted as HOSTNAME1:PORT1,HOSTNAME:PORT2,...
|
||||
|
||||
This is the format expected by many config files.
|
||||
"""
|
||||
port_mapping = self.port_mappings[protocol]
|
||||
self.logger.info("Bootstrap client port is: " + str(port_mapping.port_number))
|
||||
return self.__bootstrap_servers(port_mapping, validate, offline_nodes)
|
||||
|
||||
def controller(self):
|
||||
""" Get the controller node
|
||||
"""
|
||||
self.logger.debug("Querying zookeeper to find controller broker")
|
||||
controller_info = self.zk.query("/controller", chroot=self.zk_chroot)
|
||||
|
||||
if controller_info is None:
|
||||
raise Exception("Error finding controller info")
|
||||
|
||||
controller_info = json.loads(controller_info)
|
||||
self.logger.debug(controller_info)
|
||||
|
||||
controller_idx = int(controller_info["brokerid"])
|
||||
self.logger.info("Controller's ID: %d" % (controller_idx))
|
||||
return self.get_node(controller_idx)
|
||||
|
||||
def is_registered(self, node):
|
||||
"""
|
||||
Check whether a broker is registered in Zookeeper
|
||||
"""
|
||||
self.logger.debug("Querying zookeeper to see if broker %s is registered", str(node))
|
||||
broker_info = self.zk.query("/brokers/ids/%s" % self.idx(node), chroot=self.zk_chroot)
|
||||
self.logger.debug("Broker info: %s", broker_info)
|
||||
return broker_info is not None
|
||||
|
||||
def get_offset_shell(self, topic, partitions, max_wait_ms, offsets, time):
|
||||
node = self.nodes[0]
|
||||
|
||||
cmd = self.path.script("kafka-run-class.sh", node)
|
||||
cmd += " kafka.tools.GetOffsetShell"
|
||||
cmd += " --topic %s --broker-list %s --max-wait-ms %s --offsets %s --time %s" % (topic, self.bootstrap_servers(self.security_protocol), max_wait_ms, offsets, time)
|
||||
|
||||
if partitions:
|
||||
cmd += ' --partitions %s' % partitions
|
||||
|
||||
cmd += " 2>> %s/get_offset_shell.log" % KafkaService.PERSISTENT_ROOT
|
||||
cmd += " | tee -a %s/get_offset_shell.log &" % KafkaService.PERSISTENT_ROOT
|
||||
output = ""
|
||||
self.logger.debug(cmd)
|
||||
for line in node.account.ssh_capture(cmd):
|
||||
output += line
|
||||
self.logger.debug(output)
|
||||
return output
|
||||
|
||||
def java_class_name(self):
|
||||
return "kafka.Kafka"
|
||||
91
tests/kafkatest/services/kafka/templates/kafka.properties
Normal file
91
tests/kafkatest/services/kafka/templates/kafka.properties
Normal file
@@ -0,0 +1,91 @@
|
||||
# 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.
|
||||
|
||||
# see kafka.server.KafkaConfig for additional details and defaults
|
||||
advertised.host.name={{ node.account.hostname }}
|
||||
|
||||
|
||||
listeners={{ listeners }}
|
||||
advertised.listeners={{ advertised_listeners }}
|
||||
listener.security.protocol.map={{ listener_security_protocol_map }}
|
||||
|
||||
{% if node.version.supports_named_listeners() %}
|
||||
inter.broker.listener.name={{ interbroker_listener.name }}
|
||||
{% else %}
|
||||
security.inter.broker.protocol={{ interbroker_listener.security_protocol }}
|
||||
{% endif %}
|
||||
|
||||
{% for k, v in listener_security_config.client_listener_overrides.iteritems() %}
|
||||
{% if listener_security_config.requires_sasl_mechanism_prefix(k) %}
|
||||
listener.name.{{ security_protocol.lower() }}.{{ security_config.client_sasl_mechanism.lower() }}.{{ k }}={{ v }}
|
||||
{% else %}
|
||||
listener.name.{{ security_protocol.lower() }}.{{ k }}={{ v }}
|
||||
{% endif %}
|
||||
{% endfor %}
|
||||
|
||||
{% if interbroker_listener.name != security_protocol %}
|
||||
{% for k, v in listener_security_config.interbroker_listener_overrides.iteritems() %}
|
||||
{% if listener_security_config.requires_sasl_mechanism_prefix(k) %}
|
||||
listener.name.{{ interbroker_listener.name.lower() }}.{{ security_config.interbroker_sasl_mechanism.lower() }}.{{ k }}={{ v }}
|
||||
{% else %}
|
||||
listener.name.{{ interbroker_listener.name.lower() }}.{{ k }}={{ v }}
|
||||
{% endif %}
|
||||
{% endfor %}
|
||||
{% endif %}
|
||||
|
||||
ssl.keystore.location=/mnt/security/test.keystore.jks
|
||||
ssl.keystore.password=test-ks-passwd
|
||||
ssl.key.password=test-ks-passwd
|
||||
ssl.keystore.type=JKS
|
||||
ssl.truststore.location=/mnt/security/test.truststore.jks
|
||||
ssl.truststore.password=test-ts-passwd
|
||||
ssl.truststore.type=JKS
|
||||
ssl.endpoint.identification.algorithm=HTTPS
|
||||
# Zookeeper TLS settings
|
||||
#
|
||||
# Note that zookeeper.ssl.client.enable will be set to true or false elsewhere, as appropriate.
|
||||
# If it is false then these ZK keystore/truststore settings will have no effect. If it is true then
|
||||
# zookeeper.clientCnxnSocket will also be set elsewhere (to org.apache.zookeeper.ClientCnxnSocketNetty)
|
||||
{% if not zk.zk_tls_encrypt_only %}
|
||||
zookeeper.ssl.keystore.location=/mnt/security/test.keystore.jks
|
||||
zookeeper.ssl.keystore.password=test-ks-passwd
|
||||
{% endif %}
|
||||
zookeeper.ssl.truststore.location=/mnt/security/test.truststore.jks
|
||||
zookeeper.ssl.truststore.password=test-ts-passwd
|
||||
#
|
||||
sasl.mechanism.inter.broker.protocol={{ security_config.interbroker_sasl_mechanism }}
|
||||
sasl.enabled.mechanisms={{ ",".join(security_config.enabled_sasl_mechanisms) }}
|
||||
sasl.kerberos.service.name=kafka
|
||||
{% if authorizer_class_name is not none %}
|
||||
ssl.client.auth=required
|
||||
authorizer.class.name={{ authorizer_class_name }}
|
||||
{% endif %}
|
||||
|
||||
zookeeper.set.acl={{"true" if zk_set_acl else "false"}}
|
||||
|
||||
zookeeper.connection.timeout.ms={{ zk_connect_timeout }}
|
||||
zookeeper.session.timeout.ms={{ zk_session_timeout }}
|
||||
|
||||
{% if replica_lag is defined %}
|
||||
replica.lag.time.max.ms={{replica_lag}}
|
||||
{% endif %}
|
||||
|
||||
{% if auto_create_topics_enable is defined and auto_create_topics_enable is not none %}
|
||||
auto.create.topics.enable={{ auto_create_topics_enable }}
|
||||
{% endif %}
|
||||
offsets.topic.num.partitions={{ num_nodes }}
|
||||
offsets.topic.replication.factor={{ 3 if num_nodes > 3 else num_nodes }}
|
||||
# Set to a low, but non-zero value to exercise this path without making tests much slower
|
||||
group.initial.rebalance.delay.ms=100
|
||||
136
tests/kafkatest/services/kafka/templates/log4j.properties
Normal file
136
tests/kafkatest/services/kafka/templates/log4j.properties
Normal file
@@ -0,0 +1,136 @@
|
||||
# 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.
|
||||
|
||||
log4j.rootLogger={{ log_level|default("DEBUG") }}, stdout
|
||||
|
||||
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
|
||||
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
|
||||
# INFO level appenders
|
||||
log4j.appender.kafkaInfoAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.kafkaInfoAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.kafkaInfoAppender.File={{ log_dir }}/info/server.log
|
||||
log4j.appender.kafkaInfoAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.kafkaInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.kafkaInfoAppender.Threshold=INFO
|
||||
|
||||
log4j.appender.stateChangeInfoAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.stateChangeInfoAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.stateChangeInfoAppender.File={{ log_dir }}/info/state-change.log
|
||||
log4j.appender.stateChangeInfoAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.stateChangeInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.stateChangeInfoAppender.Threshold=INFO
|
||||
|
||||
log4j.appender.requestInfoAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.requestInfoAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.requestInfoAppender.File={{ log_dir }}/info/kafka-request.log
|
||||
log4j.appender.requestInfoAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.requestInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.requestInfoAppender.Threshold=INFO
|
||||
|
||||
log4j.appender.cleanerInfoAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.cleanerInfoAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.cleanerInfoAppender.File={{ log_dir }}/info/log-cleaner.log
|
||||
log4j.appender.cleanerInfoAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.cleanerInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.cleanerInfoAppender.Threshold=INFO
|
||||
|
||||
log4j.appender.controllerInfoAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.controllerInfoAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.controllerInfoAppender.File={{ log_dir }}/info/controller.log
|
||||
log4j.appender.controllerInfoAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.controllerInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.controllerInfoAppender.Threshold=INFO
|
||||
|
||||
log4j.appender.authorizerInfoAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.authorizerInfoAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.authorizerInfoAppender.File={{ log_dir }}/info/kafka-authorizer.log
|
||||
log4j.appender.authorizerInfoAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.authorizerInfoAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.authorizerInfoAppender.Threshold=INFO
|
||||
|
||||
# DEBUG level appenders
|
||||
log4j.appender.kafkaDebugAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.kafkaDebugAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.kafkaDebugAppender.File={{ log_dir }}/debug/server.log
|
||||
log4j.appender.kafkaDebugAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.kafkaDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.kafkaDebugAppender.Threshold=DEBUG
|
||||
|
||||
log4j.appender.stateChangeDebugAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.stateChangeDebugAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.stateChangeDebugAppender.File={{ log_dir }}/debug/state-change.log
|
||||
log4j.appender.stateChangeDebugAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.stateChangeDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.stateChangeDebugAppender.Threshold=DEBUG
|
||||
|
||||
log4j.appender.requestDebugAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.requestDebugAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.requestDebugAppender.File={{ log_dir }}/debug/kafka-request.log
|
||||
log4j.appender.requestDebugAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.requestDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.requestDebugAppender.Threshold=DEBUG
|
||||
|
||||
log4j.appender.cleanerDebugAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.cleanerDebugAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.cleanerDebugAppender.File={{ log_dir }}/debug/log-cleaner.log
|
||||
log4j.appender.cleanerDebugAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.cleanerDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.cleanerDebugAppender.Threshold=DEBUG
|
||||
|
||||
log4j.appender.controllerDebugAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.controllerDebugAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.controllerDebugAppender.File={{ log_dir }}/debug/controller.log
|
||||
log4j.appender.controllerDebugAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.controllerDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.controllerDebugAppender.Threshold=DEBUG
|
||||
|
||||
log4j.appender.authorizerDebugAppender=org.apache.log4j.DailyRollingFileAppender
|
||||
log4j.appender.authorizerDebugAppender.DatePattern='.'yyyy-MM-dd-HH
|
||||
log4j.appender.authorizerDebugAppender.File={{ log_dir }}/debug/kafka-authorizer.log
|
||||
log4j.appender.authorizerDebugAppender.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.authorizerDebugAppender.layout.ConversionPattern=[%d] %p %m (%c)%n
|
||||
log4j.appender.authorizerDebugAppender.Threshold=DEBUG
|
||||
|
||||
# Turn on all our debugging info
|
||||
log4j.logger.kafka.producer.async.DefaultEventHandler={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender
|
||||
log4j.logger.kafka.client.ClientUtils={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender
|
||||
log4j.logger.kafka.perf={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender
|
||||
log4j.logger.kafka.perf.ProducerPerformance$ProducerThread={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender
|
||||
log4j.logger.kafka={{ log_level|default("DEBUG") }}, kafkaInfoAppender, kafkaDebugAppender
|
||||
|
||||
log4j.logger.kafka.network.RequestChannel$={{ log_level|default("DEBUG") }}, requestInfoAppender, requestDebugAppender
|
||||
log4j.additivity.kafka.network.RequestChannel$=false
|
||||
|
||||
log4j.logger.kafka.network.Processor={{ log_level|default("DEBUG") }}, requestInfoAppender, requestDebugAppender
|
||||
log4j.logger.kafka.server.KafkaApis={{ log_level|default("DEBUG") }}, requestInfoAppender, requestDebugAppender
|
||||
log4j.additivity.kafka.server.KafkaApis=false
|
||||
log4j.logger.kafka.request.logger={{ log_level|default("DEBUG") }}, requestInfoAppender, requestDebugAppender
|
||||
log4j.additivity.kafka.request.logger=false
|
||||
|
||||
log4j.logger.kafka.controller={{ log_level|default("DEBUG") }}, controllerInfoAppender, controllerDebugAppender
|
||||
log4j.additivity.kafka.controller=false
|
||||
|
||||
log4j.logger.kafka.log.LogCleaner={{ log_level|default("DEBUG") }}, cleanerInfoAppender, cleanerDebugAppender
|
||||
log4j.additivity.kafka.log.LogCleaner=false
|
||||
|
||||
log4j.logger.state.change.logger={{ log_level|default("DEBUG") }}, stateChangeInfoAppender, stateChangeDebugAppender
|
||||
log4j.additivity.state.change.logger=false
|
||||
|
||||
#Change this to debug to get the actual audit log for authorizer.
|
||||
log4j.logger.kafka.authorizer.logger={{ log_level|default("DEBUG") }}, authorizerInfoAppender, authorizerDebugAppender
|
||||
log4j.additivity.kafka.authorizer.logger=false
|
||||
|
||||
18
tests/kafkatest/services/kafka/util.py
Normal file
18
tests/kafkatest/services/kafka/util.py
Normal file
@@ -0,0 +1,18 @@
|
||||
# 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.
|
||||
|
||||
from collections import namedtuple
|
||||
|
||||
TopicPartition = namedtuple('TopicPartition', ['topic', 'partition'])
|
||||
Reference in New Issue
Block a user