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 = fix_opts_for_new_jvm(node) 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 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 = fix_opts_for_new_jvm(node) cmd += "echo %s > %s && " % (json_str, json_file) cmd += "%s " % self.path.script( "kafka-reassign-partitions.sh", node) cmd += self._connect_setting_reassign_partitions(node) 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 start_cmd(self, node): cmd = fix_opts_for_new_jvm(node) cmd += self.path.script("kafka-run-class.sh", node) cmd += " " cmd += self.java_class_name() cmd += " --topic %s --broker-list %s" % ( self.topic, self.kafka.bootstrap_servers(self.security_protocol)) if self.max_messages > 0: cmd += " --max-messages %s" % str(self.max_messages) if self.security_protocol != SecurityConfig.PLAINTEXT: cmd += " --security-protocol %s" % str(self.security_protocol) if self.security_protocol == SecurityConfig.SSL or self.security_protocol == SecurityConfig.SASL_SSL: cmd += " --ssl-truststore-location %s" % str( SecurityConfig.TRUSTSTORE_PATH) cmd += " --ssl-truststore-password %s" % str( SecurityConfig.ssl_stores.truststore_passwd) if self.security_protocol == SecurityConfig.SASL_PLAINTEXT or \ self.security_protocol == SecurityConfig.SASL_SSL or \ self.security_protocol == SecurityConfig.SASL_MECHANISM_GSSAPI or \ self.security_protocol == SecurityConfig.SASL_MECHANISM_PLAIN: cmd += " --sasl-kerberos-service-name %s" % str('kafka') cmd += " --client-jaas-conf-path %s" % str( SecurityConfig.JAAS_CONF_PATH) cmd += " --kerb5-conf-path %s" % str(SecurityConfig.KRB5CONF_PATH) cmd += " 2>> /mnt/kafka_log4j_appender.log | tee -a /mnt/kafka_log4j_appender.log &" return cmd
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 = fix_opts_for_new_jvm(node) 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 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 = fix_opts_for_new_jvm(node) 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 start_cmd(self, node, idx): cmd = "export LOG_DIR=%s;" % VerifiableProducer.LOG_DIR if self.kafka_opts_override: cmd += " export KAFKA_OPTS=\"%s\";" % self.kafka_opts_override else: cmd += " export KAFKA_OPTS=%s;" % self.security_config.kafka_opts cmd += fix_opts_for_new_jvm(node) cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % VerifiableProducer.LOG4J_CONFIG cmd += self.impl.exec_cmd(node) cmd += " --topic %s --broker-list %s" % ( self.topic, self.kafka.bootstrap_servers( self.security_config.security_protocol, True, self.offline_nodes)) if self.max_messages > 0: cmd += " --max-messages %s" % str(self.max_messages) if self.throughput > 0: cmd += " --throughput %s" % str(self.throughput) if self.message_validator == is_int_with_prefix: cmd += " --value-prefix %s" % str(idx) if self.acks is not None: cmd += " --acks %s " % str(self.acks) if self.create_time > -1: cmd += " --message-create-time %s " % str(self.create_time) if self.repeating_keys is not None: cmd += " --repeating-keys %s " % str(self.repeating_keys) cmd += " --producer.config %s" % VerifiableProducer.CONFIG_FILE cmd += " 2>> %s | tee -a %s &" % (VerifiableProducer.STDOUT_CAPTURE, VerifiableProducer.STDOUT_CAPTURE) return cmd
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 = fix_opts_for_new_jvm(node) cmd += "%s %s --entity-name %s --entity-type topics --alter --add-config message.format.version=%s" % \ (self.path.script("kafka-configs.sh", node), self._connect_setting_kafka_configs(node), topic, msg_format_version) self.logger.info("Running alter message format command...\n%s" % cmd) node.account.ssh(cmd)
def list_topics(self, node=None, use_zk_to_list_topic=True): if node is None: node = self.nodes[0] cmd = fix_opts_for_new_jvm(node) 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 _acl_cmd_prefix(self, kafka, node, force_use_zk_connection): """ :param node: Node to use when determining connection settings :param force_use_zk_connection: forces the use of ZooKeeper when true, otherwise AdminClient is used when available :return command prefix for running kafka-acls """ cmd = fix_opts_for_new_jvm(node) cmd += "%s %s %s" % ( kafka.kafka_acls_cmd(node, force_use_zk_connection), self._acl_command_connect_setting(kafka, node, force_use_zk_connection), self._kafka_acls_cmd_config(kafka, node, force_use_zk_connection)) return cmd
def list_topics(self, node=None): if node is None: node = self.nodes[0] force_use_zk_connection = not self.all_nodes_topic_command_supports_bootstrap_server() cmd = fix_opts_for_new_jvm(node) cmd += "%s %s --list %s" % (self._kafka_topics_cmd(node, force_use_zk_connection), self._topic_command_connect_setting(node, force_use_zk_connection), self._kafka_topics_cmd_config(node, force_use_zk_connection)) for line in node.account.ssh_capture(cmd): if not line.startswith("SLF4J"): yield line.rstrip()
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 = fix_opts_for_new_jvm(node) cmd += "%s %s --entity-name %s --entity-type topics --alter --add-config unclean.leader.election.enable=%s" % \ (self.path.script("kafka-configs.sh", node), self._connect_setting_kafka_configs(node), topic, str(value).lower()) self.logger.info("Running alter unclean leader 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 = fix_opts_for_new_jvm(node) 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 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 += fix_opts_for_new_jvm(node) 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 exec_cmd (self, node): """ :return: command to execute to start instance Translates Verifiable* to the corresponding Java client class name """ cmd = "" if self.java_class_name == 'VerifiableProducer' and node.version <= LATEST_0_8_2: # 0.8.2.X releases do not have VerifiableProducer.java, so cheat and add # the tools jar from trunk to the classpath tools_jar = self.parent.path.jar(TOOLS_JAR_NAME, DEV_BRANCH) tools_dependant_libs_jar = self.parent.path.jar(TOOLS_DEPENDANT_TEST_LIBS_JAR_NAME, DEV_BRANCH) cmd += "for file in %s; do CLASSPATH=$CLASSPATH:$file; done; " % tools_jar cmd += "for file in %s; do CLASSPATH=$CLASSPATH:$file; done; " % tools_dependant_libs_jar cmd += "export CLASSPATH; " cmd += fix_opts_for_new_jvm(node) cmd += self.parent.path.script("kafka-run-class.sh", node) + " org.apache.kafka.tools." + self.java_class_name return cmd
def start_cmd(self, node, connector_configs): cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % self.LOG4J_CONFIG_FILE heap_kafka_opts = "-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=%s" % \ self.logs["connect_heap_dump_file"]["path"] other_kafka_opts = self.security_config.kafka_opts.strip('\"') cmd += fix_opts_for_new_jvm(node) cmd += "export KAFKA_OPTS=\"%s %s\"; " % (heap_kafka_opts, other_kafka_opts) for envvar in self.environment: cmd += "export %s=%s; " % (envvar, str(self.environment[envvar])) cmd += "%s %s " % (self.path.script("connect-standalone.sh", node), self.CONFIG_FILE) cmd += " ".join(connector_configs) cmd += " & echo $! >&3 ) 1>> %s 2>> %s 3> %s" % ( self.STDOUT_FILE, self.STDERR_FILE, self.PID_FILE) return cmd
def describe_topic(self, topic, node=None): if node is None: node = self.nodes[0] force_use_zk_connection = not self.all_nodes_topic_command_supports_bootstrap_server() cmd = fix_opts_for_new_jvm(node) cmd += "%s %s --topic %s --describe %s" % \ (self._kafka_topics_cmd(node, force_use_zk_connection), self._topic_command_connect_setting(node, force_use_zk_connection), topic, self._kafka_topics_cmd_config(node, force_use_zk_connection)) self.logger.info("Running topic describe command...\n%s" % cmd) output = "" for line in node.account.ssh_capture(cmd): output += line return output
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 = fix_opts_for_new_jvm(node) 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._topic_command_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 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 = fix_opts_for_new_jvm(node) cmd += "%s --bootstrap-server %s %s --list" % \ (consumer_group_script, self.bootstrap_servers(self.security_protocol), command_config) return self.run_cli_tool(node, cmd)
def delete_topic(self, topic, node=None, use_zk_to_delete_topic=False): """ 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) cmd = fix_opts_for_new_jvm(node) cmd += "%s %s --topic %s --delete %s" % \ (self._kafka_topics_cmd(node=node, use_zk_connection=use_zk_to_delete_topic), self._topic_command_connect_setting(node=node, use_zk_connection=use_zk_to_delete_topic), topic, self._kafka_topics_cmd_config(node=node, use_zk_connection=use_zk_to_delete_topic)) self.logger.info("Running topic delete command...\n%s" % cmd) node.account.ssh(cmd)
def get_offset_shell(self, topic, partitions, max_wait_ms, offsets, time): node = self.nodes[0] cmd = fix_opts_for_new_jvm(node) 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 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) force_use_zk_connection = not self.all_nodes_topic_command_supports_bootstrap_server() cmd = fix_opts_for_new_jvm(node) cmd += "%s %s --topic %s --delete %s" % \ (self._kafka_topics_cmd(node, force_use_zk_connection), self._topic_command_connect_setting(node, force_use_zk_connection), topic, self._kafka_topics_cmd_config(node, force_use_zk_connection)) self.logger.info("Running topic delete 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 = fix_opts_for_new_jvm(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 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 = fix_opts_for_new_jvm(node) cmd += "echo %s > %s && " % (json_str, json_file) cmd += "%s " % self.path.script("kafka-reassign-partitions.sh", node) cmd += self._connect_setting_reassign_partitions(node) 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 start_cmd(self, node): """Return the start command appropriate for the given node.""" args = self.args.copy() args['broker_list'] = self.kafka.bootstrap_servers( self.security_config.security_protocol) if not self.new_consumer: args['zk_connect'] = self.kafka.zk_connect_setting() args['stdout'] = ConsoleConsumer.STDOUT_CAPTURE args['stderr'] = ConsoleConsumer.STDERR_CAPTURE args['log_dir'] = ConsoleConsumer.LOG_DIR args['log4j_config'] = ConsoleConsumer.LOG4J_CONFIG args['config_file'] = ConsoleConsumer.CONFIG_FILE args['stdout'] = ConsoleConsumer.STDOUT_CAPTURE args['jmx_port'] = self.jmx_port args['console_consumer'] = self.path.script( "kafka-console-consumer.sh", node) if self.kafka_opts_override: args['kafka_opts'] = "\"%s\"" % self.kafka_opts_override else: args['kafka_opts'] = self.security_config.kafka_opts cmd = fix_opts_for_new_jvm(node) cmd += "export JMX_PORT=%(jmx_port)s; " \ "export LOG_DIR=%(log_dir)s; " \ "export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j_config)s\"; " \ "export KAFKA_OPTS=%(kafka_opts)s; " \ "%(console_consumer)s " \ "--topic %(topic)s " \ "--consumer.config %(config_file)s " % args if self.new_consumer: assert node.version.consumer_supports_bootstrap_server(), \ "new_consumer is only supported if version >= 0.9.0.0, version %s" % str(node.version) if node.version <= LATEST_0_10_0: cmd += " --new-consumer" cmd += " --bootstrap-server %(broker_list)s" % args if node.version >= V_0_11_0_0: cmd += " --isolation-level %s" % self.isolation_level else: assert node.version < V_2_0_0, \ "new_consumer==false is only supported if version < 2.0.0, version %s" % str(node.version) cmd += " --zookeeper %(zk_connect)s" % args if self.from_beginning: cmd += " --from-beginning" if self.consumer_timeout_ms is not None: # version 0.8.X and below do not support --timeout-ms option # This will be added in the properties file instead if node.version > LATEST_0_8_2: cmd += " --timeout-ms %s" % self.consumer_timeout_ms if self.print_timestamp: cmd += " --property print.timestamp=true" if self.print_key: cmd += " --property print.key=true" if self.print_partition: cmd += " --property print.partition=true" # LoggingMessageFormatter was introduced after 0.9 if node.version > LATEST_0_9: cmd += " --formatter kafka.tools.LoggingMessageFormatter" if self.enable_systest_events: # enable systest events is only available in 0.10.0 and later # check the assertion here as well, in case node.version has been modified assert node.version >= V_0_10_0_0 cmd += " --enable-systest-events" if self.consumer_properties is not None: for k, v in self.consumer_properties.items(): cmd += " --consumer-property %s=%s" % (k, v) cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args return cmd