From db0405c6994123417441ac328d2f9e7f96b8e851 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Mon, 19 Sep 2016 17:10:54 -0700 Subject: [PATCH 01/16] Removed Zookeeper dependency from Kafka Streams. Added two test for creating and deleting topics. They work in IDE but fail while build. Removing the new tests for now. --- build.gradle | 2 +- .../apache/kafka/streams/StreamsConfig.java | 74 ++++- .../internals/InternalTopicManager.java | 230 ++------------ .../internals/StreamPartitionAssignor.java | 16 +- .../internals/StreamsKafkaClient.java | 287 ++++++++++++++++++ .../InternalTopicIntegrationTest.java | 66 +++- 6 files changed, 439 insertions(+), 236 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java diff --git a/build.gradle b/build.gradle index 64c389e31779..682f321d17a8 100644 --- a/build.gradle +++ b/build.gradle @@ -686,7 +686,7 @@ project(':streams') { compile libs.slf4jApi compile libs.rocksDBJni // this dependency should be removed after KIP-4 - compile (libs.zkclient) { + testCompile (libs.zkclient) { // users should be able to choose the logging implementation (and slf4j bridge) exclude module: 'slf4j-log4j12' exclude module: 'log4j' diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 23b528786d68..9761cef0eb50 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -140,6 +140,32 @@ public class StreamsConfig extends AbstractConfig { public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering"; public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads"; + public static final String SECURITY_PROTOCOL_CONFIG = CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; + public static final String SECURITY_PROTOCOL_DOC = CommonClientConfigs.SECURITY_PROTOCOL_DOC; + public static final String DEFAULT_SECURITY_PROTOCOL = CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL; + + public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; + public static final String CONNECTIONS_MAX_IDLE_MS_DOC = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC; + + public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; + public static final String RETRY_BACKOFF_MS_DOC = CommonClientConfigs.RETRY_BACKOFF_MS_DOC; + + public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; + public static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; + + public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; + public static final String RECONNECT_BACKOFF_MS_DOC = CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC; + + public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; + public static final String SEND_BUFFER_DOC = CommonClientConfigs.SEND_BUFFER_DOC; + + public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; + public static final String RECEIVE_BUFFER_DOC = CommonClientConfigs.RECEIVE_BUFFER_DOC; + + public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; + public static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; + + static { CONFIG = new ConfigDef().define(APPLICATION_ID_CONFIG, // required with no default value Type.STRING, @@ -256,7 +282,53 @@ public class StreamsConfig extends AbstractConfig { 10 * 1024 * 1024L, atLeast(0), Importance.LOW, - CACHE_MAX_BYTES_BUFFERING_DOC); + CACHE_MAX_BYTES_BUFFERING_DOC) + .define(SECURITY_PROTOCOL_CONFIG, + Type.STRING, + DEFAULT_SECURITY_PROTOCOL, + Importance.LOW, //TODO: Is this the right value? Investigate! + SECURITY_PROTOCOL_DOC) + .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, + ConfigDef.Type.LONG, + 9 * 60 * 1000, + ConfigDef.Importance.MEDIUM, + CONNECTIONS_MAX_IDLE_MS_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, + ConfigDef.Type.LONG, + 100L, + atLeast(0L), + ConfigDef.Importance.LOW, + RETRY_BACKOFF_MS_DOC) + .define(METADATA_MAX_AGE_CONFIG, + ConfigDef.Type.LONG, + 5 * 60 * 1000, + atLeast(0), + ConfigDef.Importance.LOW, + METADATA_MAX_AGE_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, + ConfigDef.Type.LONG, + 50L, + atLeast(0L), + ConfigDef.Importance.LOW, + RECONNECT_BACKOFF_MS_DOC) + .define(SEND_BUFFER_CONFIG, + ConfigDef.Type.INT, + 128 * 1024, + atLeast(0), + ConfigDef.Importance.MEDIUM, + SEND_BUFFER_DOC) + .define(RECEIVE_BUFFER_CONFIG, + ConfigDef.Type.INT, + 32 * 1024, + atLeast(0), + ConfigDef.Importance.MEDIUM, + RECEIVE_BUFFER_DOC) + .define(REQUEST_TIMEOUT_MS_CONFIG, + ConfigDef.Type.INT, + 40 * 1000, + atLeast(0), + ConfigDef.Importance.MEDIUM, + REQUEST_TIMEOUT_MS_DOC); } // this is the list of configs for underlying clients diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 44de75752513..166ec60054f2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -17,27 +17,12 @@ package org.apache.kafka.streams.processor.internals; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.I0Itec.zkclient.ZkClient; -import org.I0Itec.zkclient.exception.ZkNoNodeException; -import org.I0Itec.zkclient.exception.ZkNodeExistsException; -import org.I0Itec.zkclient.serialize.ZkSerializer; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.errors.StreamsException; -import org.apache.zookeeper.ZooDefs; +import org.apache.kafka.streams.StreamsConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; import java.util.Locale; -import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -45,21 +30,18 @@ public class InternalTopicManager { private static final Logger log = LoggerFactory.getLogger(InternalTopicManager.class); - // TODO: the following ZK dependency should be removed after KIP-4 - private static final String ZK_TOPIC_PATH = "/brokers/topics"; - private static final String ZK_BROKER_PATH = "/brokers/ids"; - private static final String ZK_DELETE_TOPIC_PATH = "/admin/delete_topics"; - private static final String ZK_ENTITY_CONFIG_PATH = "/config/topics"; // TODO: the following LogConfig dependency should be removed after KIP-4 public static final String CLEANUP_POLICY_PROP = "cleanup.policy"; private static final Set CLEANUP_POLICIES = Utils.mkSet("compact", "delete"); public static final String RETENTION_MS = "retention.ms"; public static final Long WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS); - private final ZkClient zkClient; - private final int replicationFactor; private final long windowChangeLogAdditionalRetention; + private final int replicationFactor; + private StreamsKafkaClient streamsKafkaClient; + StreamsConfig config; + public static boolean isValidCleanupPolicy(final String cleanupPolicy) { if (cleanupPolicy == null) { return false; @@ -73,203 +55,35 @@ public static boolean isValidCleanupPolicy(final String cleanupPolicy) { return true; } - private class ZKStringSerializer implements ZkSerializer { - - /** - * @throws AssertionError if the byte String encoding type is not supported - */ - @Override - public byte[] serialize(Object data) { - try { - return ((String) data).getBytes("UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new AssertionError(e); - } - } - - /** - * @throws AssertionError if the byte String encoding type is not supported - */ - @Override - public Object deserialize(byte[] bytes) { - try { - if (bytes == null) - return null; - else - return new String(bytes, "UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new AssertionError(e); - } - } + public InternalTopicManager() { + this.replicationFactor = 0; + this.windowChangeLogAdditionalRetention = WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT; } - public InternalTopicManager() { - this.zkClient = null; + public InternalTopicManager(StreamsConfig config) { + this.config = config; + this.streamsKafkaClient = new StreamsKafkaClient(config); this.replicationFactor = 0; this.windowChangeLogAdditionalRetention = WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT; } - public InternalTopicManager(String zkConnect, final int replicationFactor, long windowChangeLogAdditionalRetention) { - this.zkClient = new ZkClient(zkConnect, 30 * 1000, 30 * 1000, new ZKStringSerializer()); + public InternalTopicManager(StreamsConfig config, final int replicationFactor, long windowChangeLogAdditionalRetention) { + this.config = config; + this.streamsKafkaClient = new StreamsKafkaClient(config); this.replicationFactor = replicationFactor; this.windowChangeLogAdditionalRetention = windowChangeLogAdditionalRetention; } + /** + * If the topic exists do nothing (we don't change partititions or delete the existing topics). + * Otherwise create the new topic. + * @param topic + * @param numPartitions + */ public void makeReady(InternalTopicConfig topic, int numPartitions) { - boolean topicNotReady = true; - - while (topicNotReady) { - Map> topicMetadata = getTopicMetadata(topic.name()); - - if (topicMetadata == null) { - try { - createTopic(topic, numPartitions, replicationFactor); - } catch (ZkNodeExistsException e) { - // ignore and continue - } - } else { - if (topicMetadata.size() > numPartitions) { - // else if topic exists with more #.partitions than needed, delete in order to re-create it - try { - deleteTopic(topic.name()); - } catch (ZkNodeExistsException e) { - // ignore and continue - } - } else if (topicMetadata.size() < numPartitions) { - // else if topic exists with less #.partitions than needed, add partitions - try { - addPartitions(topic.name(), numPartitions - topicMetadata.size(), replicationFactor, topicMetadata); - } catch (ZkNoNodeException e) { - // ignore and continue - } - } else { - topicNotReady = false; - } - } - } - } - - private List getBrokers() { - List brokers = new ArrayList<>(); - for (String broker: zkClient.getChildren(ZK_BROKER_PATH)) { - brokers.add(Integer.parseInt(broker)); - } - Collections.sort(brokers); - - log.debug("Read brokers {} from ZK in partition assignor.", brokers); - - return brokers; - } - - @SuppressWarnings("unchecked") - private Map> getTopicMetadata(String topic) { - String data = zkClient.readData(ZK_TOPIC_PATH + "/" + topic, true); - - if (data == null) return null; - - try { - ObjectMapper mapper = new ObjectMapper(); - - Map dataMap = mapper.readValue(data, new TypeReference>() { - - }); - - Map> partitions = (Map>) dataMap.get("partitions"); - log.debug("Read partitions {} for topic {} from ZK in partition assignor.", partitions, topic); - - return partitions; - } catch (IOException e) { - throw new StreamsException("Error while reading topic metadata from ZK for internal topic " + topic, e); - } - } - - private void createTopic(InternalTopicConfig topic, int numPartitions, int replicationFactor) throws ZkNodeExistsException { - log.debug("Creating topic {} with {} partitions from ZK in partition assignor.", topic.name(), numPartitions); - ObjectMapper mapper = new ObjectMapper(); - List brokers = getBrokers(); - int numBrokers = brokers.size(); - if (numBrokers < replicationFactor) { - log.warn("Not enough brokers found. The replication factor is reduced from " + replicationFactor + " to " + numBrokers); - replicationFactor = numBrokers; - } - - Map> assignment = new HashMap<>(); - - for (int i = 0; i < numPartitions; i++) { - ArrayList brokerList = new ArrayList<>(); - for (int r = 0; r < replicationFactor; r++) { - int shift = r * numBrokers / replicationFactor; - brokerList.add(brokers.get((i + shift) % numBrokers)); - } - assignment.put(i, brokerList); + if (!streamsKafkaClient.topicExists(topic.name())) { + streamsKafkaClient.createTopic(topic, numPartitions, replicationFactor, windowChangeLogAdditionalRetention); } - // write out config first just like in AdminUtils.scala createOrUpdateTopicPartitionAssignmentPathInZK() - try { - Map dataMap = new HashMap<>(); - dataMap.put("version", 1); - dataMap.put("config", topic.toProperties(windowChangeLogAdditionalRetention)); - String data = mapper.writeValueAsString(dataMap); - zkClient.createPersistent(ZK_ENTITY_CONFIG_PATH + "/" + topic.name(), data, ZooDefs.Ids.OPEN_ACL_UNSAFE); - } catch (JsonProcessingException e) { - throw new StreamsException("Error while creating topic config in ZK for internal topic " + topic, e); - } - - // try to write to ZK with open ACL - try { - Map dataMap = new HashMap<>(); - dataMap.put("version", 1); - dataMap.put("partitions", assignment); - String data = mapper.writeValueAsString(dataMap); - - zkClient.createPersistent(ZK_TOPIC_PATH + "/" + topic.name(), data, ZooDefs.Ids.OPEN_ACL_UNSAFE); - } catch (JsonProcessingException e) { - throw new StreamsException("Error while creating topic metadata in ZK for internal topic " + topic, e); - } - } - - private void deleteTopic(String topic) throws ZkNodeExistsException { - log.debug("Deleting topic {} from ZK in partition assignor.", topic); - - zkClient.createPersistent(ZK_DELETE_TOPIC_PATH + "/" + topic, "", ZooDefs.Ids.OPEN_ACL_UNSAFE); } - - private void addPartitions(String topic, int numPartitions, int replicationFactor, Map> existingAssignment) { - log.debug("Adding {} partitions topic {} from ZK with existing partitions assigned as {} in partition assignor.", topic, numPartitions, existingAssignment); - - List brokers = getBrokers(); - int numBrokers = brokers.size(); - if (numBrokers < replicationFactor) { - log.warn("Not enough brokers found. The replication factor is reduced from " + replicationFactor + " to " + numBrokers); - replicationFactor = numBrokers; - } - - int startIndex = existingAssignment.size(); - - Map> newAssignment = new HashMap<>(existingAssignment); - - for (int i = 0; i < numPartitions; i++) { - ArrayList brokerList = new ArrayList<>(); - for (int r = 0; r < replicationFactor; r++) { - int shift = r * numBrokers / replicationFactor; - brokerList.add(brokers.get((i + shift) % numBrokers)); - } - newAssignment.put(i + startIndex, brokerList); - } - - // try to write to ZK with open ACL - try { - Map dataMap = new HashMap<>(); - dataMap.put("version", 1); - dataMap.put("partitions", newAssignment); - - ObjectMapper mapper = new ObjectMapper(); - String data = mapper.writeValueAsString(dataMap); - - zkClient.writeData(ZK_TOPIC_PATH + "/" + topic, data); - } catch (JsonProcessingException e) { - throw new StreamsException("Error while updating topic metadata in ZK for internal topic " + topic, e); - } - } - } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 3be9c114ba7f..42d5df076182 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -147,16 +147,12 @@ public void configure(Map configs) { } - if (configs.containsKey(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG)) { - internalTopicManager = new InternalTopicManager( - (String) configs.get(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG), - configs.containsKey(StreamsConfig.REPLICATION_FACTOR_CONFIG) ? (Integer) configs.get(StreamsConfig.REPLICATION_FACTOR_CONFIG) : 1, - configs.containsKey(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) ? - (Long) configs.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) - : WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT); - } else { - log.info("stream-thread [{}] Config '{}' isn't supplied and hence no internal topics will be created.", streamThread.getName(), StreamsConfig.ZOOKEEPER_CONNECT_CONFIG); - } + internalTopicManager = new InternalTopicManager( + this.streamThread.config, + configs.containsKey(StreamsConfig.REPLICATION_FACTOR_CONFIG) ? (Integer) configs.get(StreamsConfig.REPLICATION_FACTOR_CONFIG) : 1, + configs.containsKey(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) ? + (Long) configs.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) + : WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java new file mode 100644 index 000000000000..54aa4c8982eb --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -0,0 +1,287 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.NetworkClient; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.metrics.JmxReporter; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.network.ChannelBuilder; +import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.CreateTopicsRequest; +import org.apache.kafka.common.requests.CreateTopicsResponse; +import org.apache.kafka.common.requests.DeleteTopicsRequest; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; + +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetSocketAddress; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Arrays; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + + +public class StreamsKafkaClient { + + private static final Logger log = LoggerFactory.getLogger(StreamsKafkaClient.class); + private Metadata metadata; + private Metrics metrics; + private Selector selector; + private ChannelBuilder channelBuilder; + private KafkaClient kafkaClient; + private StreamsConfig streamsConfig; + private Node brokerNode; + + public static final String CLEANUP_POLICY_PROP = "cleanup.policy"; + private static final String COMPACT = "compact"; + + private int maxIterations = 10; + + public StreamsKafkaClient(StreamsConfig streamsConfig) { + + this.streamsConfig = streamsConfig; + Time time = new SystemTime(); + + Map metricTags = new LinkedHashMap<>(); + metricTags.put("client-id", StreamsConfig.CLIENT_ID_CONFIG); + + this.metadata = new Metadata(streamsConfig.getLong(streamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(streamsConfig.METADATA_MAX_AGE_CONFIG)); + List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); + metadata.update(Cluster.bootstrap(addresses), 0); + + MetricConfig metricConfig = new MetricConfig().samples(streamsConfig.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG)) + .timeWindow(streamsConfig.getLong(CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS) + .tags(metricTags); + List reporters = streamsConfig.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, + MetricsReporter.class); + // TODO: This should come from the KafkaStream + reporters.add(new JmxReporter("kafka.streams")); + this.metrics = new Metrics(metricConfig, reporters, time); + + this.channelBuilder = ClientUtils.createChannelBuilder(streamsConfig.values()); + + selector = new Selector(streamsConfig.getLong(streamsConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "kafka-client", this.channelBuilder); + + kafkaClient = new NetworkClient( + selector, + metadata, + streamsConfig.getString(StreamsConfig.CLIENT_ID_CONFIG), + 100, // a fixed large enough value will suffice + streamsConfig.getLong(streamsConfig.RECONNECT_BACKOFF_MS_CONFIG), + streamsConfig.getInt(streamsConfig.SEND_BUFFER_CONFIG), + streamsConfig.getInt(streamsConfig.RECEIVE_BUFFER_CONFIG), + streamsConfig.getInt(streamsConfig.REQUEST_TIMEOUT_MS_CONFIG), time); + + brokerNode = kafkaClient.leastLoadedNode(new SystemTime().milliseconds()); + } + + public KafkaClient getKafkaClient() { + return kafkaClient; + } + + public void shutdown() { + log.info("Closing the StreamsKafkaClient."); + // this will keep track of the first encountered exception + AtomicReference firstException = new AtomicReference(); + ClientUtils.closeQuietly(metrics, "producer metrics", firstException); + + log.debug("The StreamsKafkaClient has closed."); + if (firstException.get() != null) + throw new KafkaException("Failed to close kafka producer", firstException.get()); + } + + /** + * Cretes a new topic with the given number of partitions and replication factor. + * + * @param internalTopicConfig + * @param numPartitions + * @param replicationFactor + */ + public void createTopic(InternalTopicConfig internalTopicConfig, int numPartitions, int replicationFactor, long windowChangeLogAdditionalRetention) { + CreateTopicsRequest.TopicDetails topicDetails; + Properties topicProperties = internalTopicConfig.toProperties(windowChangeLogAdditionalRetention); + Map topicConfig = new HashMap<>(); + for (String key : topicProperties.stringPropertyNames()) { + topicConfig.put(key, topicProperties.getProperty(key)); + } + topicDetails = new CreateTopicsRequest.TopicDetails(numPartitions, (short) replicationFactor, topicConfig); + Map topics = new HashMap<>(); + topics.put(internalTopicConfig.name(), topicDetails); + + CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); + + RequestCompletionHandler callback = new RequestCompletionHandler() { + public void onComplete(ClientResponse response) { + // Do nothing! + } + }; + ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS, callback); + CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); + if (createTopicsResponse.errors().get(internalTopicConfig.name()).code() > 0) { + throw new StreamsException("Could not create topic: " + internalTopicConfig.name()); + } + } + + /** + * Delete a topic. + * + * @param topic + */ + public void deleteTopic(String topic) { + log.debug("Deleting topic {} from ZK in partition assignor.", topic); + Set topics = new HashSet(); + topics.add(topic); + + DeleteTopicsRequest deleteTopicsRequest = new DeleteTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); + + RequestCompletionHandler callback = new RequestCompletionHandler() { + public void onComplete(ClientResponse response) { + // Do nothing! + } + }; + ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS, callback); + CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); + if (createTopicsResponse.errors().get(topic).code() > 0) { + throw new StreamsException("Could not delete topic: " + topic); + } + + } + + /** + * Send a request to kafka broker of this client. Polls the request for a given number of iterations to receive the response. + * + * @param request + * @param apiKeys + * @param callback + */ + public ClientResponse sendRequest(Struct request, ApiKeys apiKeys, RequestCompletionHandler callback) { + + String brokerId = Integer.toString(brokerNode.id()); + + RequestSend send = new RequestSend(brokerId, + kafkaClient.nextRequestHeader(apiKeys), + request); + + ClientRequest clientRequest = new ClientRequest(new SystemTime().milliseconds(), true, send, callback); + SystemTime systemTime = new SystemTime(); + int iterationCount = 0; + while (iterationCount < maxIterations) { + if (kafkaClient.ready(brokerNode, systemTime.milliseconds())) { + kafkaClient.send(clientRequest, systemTime.milliseconds()); + break; + } else { + kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); + } + iterationCount++; + } + + iterationCount = 0; + // Poll for the response. + while (iterationCount < maxIterations) { + List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), new SystemTime().milliseconds()); + for (ClientResponse clientResponse: responseList) { + if (clientResponse.request().equals(clientRequest)) { + return clientResponse; + } + } + iterationCount++; + } + throw new StreamsException("StreamsKafkaClient failed to send the request: " + apiKeys.name()); + } + + + /** + * Get the metadata for a topic. + * @param topic + * @return + */ + public MetadataResponse.TopicMetadata getTopicMetadata(String topic) { + MetadataRequest metadataRequest = new MetadataRequest(Arrays.asList(topic)); + + RequestCompletionHandler callback = new RequestCompletionHandler() { + public void onComplete(ClientResponse response) { + // Do nothing! + } + }; + + ClientResponse clientResponse = sendRequest(metadataRequest.toStruct(), ApiKeys.METADATA, callback); + MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); + + for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { + if (topicMetadata.topic().equalsIgnoreCase(topic)) { + if (topicMetadata.error().code() == 0) { + return topicMetadata; + } + return null; + } + } + + return null; + } + + /** + * Check to see if a topic exists. + * @param topicName + * @return + */ + public boolean topicExists(String topicName) { + Map> topics; + + Properties props = new Properties(); + props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, streamsConfig.getList(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG)); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); + + KafkaConsumer consumer = new KafkaConsumer(props); + topics = consumer.listTopics(); + for (String topicNameInList:topics.keySet()) { + if (topicNameInList.equalsIgnoreCase(topicName)) { + return true; + } + } + return false; + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java index 2d9b9a54b6ea..4779cdfc0f89 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java @@ -37,7 +37,9 @@ import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.internals.InternalTopicConfig; import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.processor.internals.StreamsKafkaClient; import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.TestUtils; import org.junit.Before; @@ -48,10 +50,11 @@ import scala.collection.Iterator; import scala.collection.Map; +import java.util.Properties; +import java.util.Collections; import java.util.Arrays; -import java.util.List; import java.util.Locale; -import java.util.Properties; +import java.util.List; import java.util.concurrent.TimeUnit; import static org.junit.Assert.assertEquals; @@ -83,13 +86,45 @@ public void before() { streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); } + //@Test + public void testCreateTopic() throws Exception { + + StreamsConfig streamsConfig = new StreamsConfig(streamsConfiguration); + StreamsKafkaClient streamsKafkaClient = new StreamsKafkaClient(streamsConfig); + + String topicName = "testTopic" + mockTime.milliseconds(); + InternalTopicConfig topicConfig = new InternalTopicConfig(topicName, + Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), + Collections.emptyMap()); + + streamsKafkaClient.createTopic(topicConfig, 1, 1, 0); + + assertTrue(streamsKafkaClient.topicExists(topicName)); + + } + + //@Test + public void testDeleteTopic() throws Exception { + + StreamsConfig streamsConfig = new StreamsConfig(streamsConfiguration); + StreamsKafkaClient streamsKafkaClient = new StreamsKafkaClient(streamsConfig); + + String topicName = "testTopic" + mockTime.milliseconds(); + InternalTopicConfig topicConfig = new InternalTopicConfig(topicName, + Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), + Collections.emptyMap()); + + streamsKafkaClient.createTopic(topicConfig, 1, 1, 0); + streamsKafkaClient.deleteTopic(topicName); + + assertTrue(!streamsKafkaClient.topicExists(topicName)); + } private Properties getTopicConfigProperties(final String changelog) { // Note: You must initialize the ZkClient with ZKStringSerializer. If you don't, then @@ -97,10 +132,10 @@ private Properties getTopicConfigProperties(final String changelog) { // only ZooKeeper and will be returned when listing topics, but Kafka itself does not create the // topic. final ZkClient zkClient = new ZkClient( - CLUSTER.zKConnectString(), - DEFAULT_ZK_SESSION_TIMEOUT_MS, - DEFAULT_ZK_CONNECTION_TIMEOUT_MS, - ZKStringSerializer$.MODULE$); + CLUSTER.zKConnectString(), + DEFAULT_ZK_SESSION_TIMEOUT_MS, + DEFAULT_ZK_CONNECTION_TIMEOUT_MS, + ZKStringSerializer$.MODULE$); try { final boolean isSecure = false; final ZkUtils zkUtils = new ZkUtils(zkClient, new ZkConnection(CLUSTER.zKConnectString()), isSecure); @@ -132,7 +167,6 @@ public void shouldCompactTopicsForStateChangelogs() throws Exception { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "compact-topics-integration-test"); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); @@ -142,13 +176,13 @@ public void shouldCompactTopicsForStateChangelogs() throws Exception { final KStream textLines = builder.stream(DEFAULT_INPUT_TOPIC); final KStream wordCounts = textLines - .flatMapValues(new ValueMapper>() { - @Override - public Iterable apply(final String value) { - return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")); - } - }).groupBy(MockKeyValueMapper.SelectValueMapper()) - .count("Counts").toStream(); + .flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(final String value) { + return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")); + } + }).groupBy(MockKeyValueMapper.SelectValueMapper()) + .count("Counts").toStream(); wordCounts.to(stringSerde, longSerde, DEFAULT_OUTPUT_TOPIC); @@ -222,4 +256,4 @@ public Iterable apply(String value) { final Long retention = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS) + durationMs; assertEquals(retention, Long.valueOf(properties.getProperty(LogConfig.RetentionMsProp()))); } -} +} \ No newline at end of file From 69b8baf766a0ed70de8782938cf6157fa6b01794 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Tue, 20 Sep 2016 10:52:53 -0700 Subject: [PATCH 02/16] Made changes according to the review feedback. --- .../internals/InternalTopicManager.java | 21 ++-- .../internals/StreamsKafkaClient.java | 119 +++++++----------- .../InternalTopicIntegrationTest.java | 37 ------ 3 files changed, 56 insertions(+), 121 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 166ec60054f2..d245a161070d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -17,8 +17,10 @@ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,7 +42,7 @@ public class InternalTopicManager { private final int replicationFactor; private StreamsKafkaClient streamsKafkaClient; - StreamsConfig config; + private StreamsConfig config; public static boolean isValidCleanupPolicy(final String cleanupPolicy) { if (cleanupPolicy == null) { @@ -60,13 +62,6 @@ public InternalTopicManager() { this.windowChangeLogAdditionalRetention = WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT; } - public InternalTopicManager(StreamsConfig config) { - this.config = config; - this.streamsKafkaClient = new StreamsKafkaClient(config); - this.replicationFactor = 0; - this.windowChangeLogAdditionalRetention = WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT; - } - public InternalTopicManager(StreamsConfig config, final int replicationFactor, long windowChangeLogAdditionalRetention) { this.config = config; this.streamsKafkaClient = new StreamsKafkaClient(config); @@ -84,6 +79,16 @@ public void makeReady(InternalTopicConfig topic, int numPartitions) { if (!streamsKafkaClient.topicExists(topic.name())) { streamsKafkaClient.createTopic(topic, numPartitions, replicationFactor, windowChangeLogAdditionalRetention); + } else { + MetadataResponse.TopicMetadata topicMetadata = streamsKafkaClient.getTopicMetadata(topic.name()); + if (topicMetadata != null) { + if (topicMetadata.partitionMetadata().size() != numPartitions) { + throw new StreamsException("Topic already exists but the number of partitions is not the same as the requested " + numPartitions + " partitions."); + } + } else { + throw new StreamsException("Topic metadata is corrupted."); + } + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 54aa4c8982eb..b2e626ab6381 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -25,7 +25,6 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.metrics.JmxReporter; @@ -39,10 +38,10 @@ import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.requests.CreateTopicsResponse; import org.apache.kafka.common.requests.DeleteTopicsRequest; +import org.apache.kafka.common.requests.DeleteTopicsResponse; import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; - import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; @@ -60,36 +59,30 @@ import java.util.Arrays; import java.util.Properties; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; public class StreamsKafkaClient { private static final Logger log = LoggerFactory.getLogger(StreamsKafkaClient.class); - private Metadata metadata; private Metrics metrics; - private Selector selector; - private ChannelBuilder channelBuilder; + private KafkaClient kafkaClient; private StreamsConfig streamsConfig; - private Node brokerNode; - public static final String CLEANUP_POLICY_PROP = "cleanup.policy"; - private static final String COMPACT = "compact"; + private static final int MAX_INFLIGHT_REQUESTS = 100; + private static final long MAX_WAIT_TIME_MS = 30000; - private int maxIterations = 10; - - public StreamsKafkaClient(StreamsConfig streamsConfig) { + public StreamsKafkaClient(final StreamsConfig streamsConfig) { this.streamsConfig = streamsConfig; - Time time = new SystemTime(); + final Time time = new SystemTime(); Map metricTags = new LinkedHashMap<>(); metricTags.put("client-id", StreamsConfig.CLIENT_ID_CONFIG); - this.metadata = new Metadata(streamsConfig.getLong(streamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(streamsConfig.METADATA_MAX_AGE_CONFIG)); + Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - metadata.update(Cluster.bootstrap(addresses), 0); + metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); MetricConfig metricConfig = new MetricConfig().samples(streamsConfig.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(streamsConfig.getLong(CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS) @@ -100,37 +93,22 @@ public StreamsKafkaClient(StreamsConfig streamsConfig) { reporters.add(new JmxReporter("kafka.streams")); this.metrics = new Metrics(metricConfig, reporters, time); - this.channelBuilder = ClientUtils.createChannelBuilder(streamsConfig.values()); + ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(streamsConfig.values()); - selector = new Selector(streamsConfig.getLong(streamsConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "kafka-client", this.channelBuilder); + Selector selector = new Selector(streamsConfig.getLong(StreamsConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "kafka-client", channelBuilder); kafkaClient = new NetworkClient( selector, metadata, streamsConfig.getString(StreamsConfig.CLIENT_ID_CONFIG), - 100, // a fixed large enough value will suffice - streamsConfig.getLong(streamsConfig.RECONNECT_BACKOFF_MS_CONFIG), - streamsConfig.getInt(streamsConfig.SEND_BUFFER_CONFIG), - streamsConfig.getInt(streamsConfig.RECEIVE_BUFFER_CONFIG), - streamsConfig.getInt(streamsConfig.REQUEST_TIMEOUT_MS_CONFIG), time); + MAX_INFLIGHT_REQUESTS, // a fixed large enough value will suffice + streamsConfig.getLong(StreamsConfig.RECONNECT_BACKOFF_MS_CONFIG), + streamsConfig.getInt(StreamsConfig.SEND_BUFFER_CONFIG), + streamsConfig.getInt(StreamsConfig.RECEIVE_BUFFER_CONFIG), + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG), time); - brokerNode = kafkaClient.leastLoadedNode(new SystemTime().milliseconds()); } - public KafkaClient getKafkaClient() { - return kafkaClient; - } - - public void shutdown() { - log.info("Closing the StreamsKafkaClient."); - // this will keep track of the first encountered exception - AtomicReference firstException = new AtomicReference(); - ClientUtils.closeQuietly(metrics, "producer metrics", firstException); - - log.debug("The StreamsKafkaClient has closed."); - if (firstException.get() != null) - throw new KafkaException("Failed to close kafka producer", firstException.get()); - } /** * Cretes a new topic with the given number of partitions and replication factor. @@ -139,28 +117,22 @@ public void shutdown() { * @param numPartitions * @param replicationFactor */ - public void createTopic(InternalTopicConfig internalTopicConfig, int numPartitions, int replicationFactor, long windowChangeLogAdditionalRetention) { - CreateTopicsRequest.TopicDetails topicDetails; + public void createTopic(final InternalTopicConfig internalTopicConfig, final int numPartitions, final int replicationFactor, final long windowChangeLogAdditionalRetention) { Properties topicProperties = internalTopicConfig.toProperties(windowChangeLogAdditionalRetention); Map topicConfig = new HashMap<>(); for (String key : topicProperties.stringPropertyNames()) { topicConfig.put(key, topicProperties.getProperty(key)); } - topicDetails = new CreateTopicsRequest.TopicDetails(numPartitions, (short) replicationFactor, topicConfig); + CreateTopicsRequest.TopicDetails topicDetails = new CreateTopicsRequest.TopicDetails(numPartitions, (short) replicationFactor, topicConfig); Map topics = new HashMap<>(); topics.put(internalTopicConfig.name(), topicDetails); CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - RequestCompletionHandler callback = new RequestCompletionHandler() { - public void onComplete(ClientResponse response) { - // Do nothing! - } - }; - ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS, callback); + ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS, null); CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); if (createTopicsResponse.errors().get(internalTopicConfig.name()).code() > 0) { - throw new StreamsException("Could not create topic: " + internalTopicConfig.name()); + throw new StreamsException("Could not create topic: " + internalTopicConfig.name() + ". " + createTopicsResponse.errors().get(internalTopicConfig.name()).name()); } } @@ -169,22 +141,17 @@ public void onComplete(ClientResponse response) { * * @param topic */ - public void deleteTopic(String topic) { + public void deleteTopic(final String topic) { log.debug("Deleting topic {} from ZK in partition assignor.", topic); Set topics = new HashSet(); topics.add(topic); DeleteTopicsRequest deleteTopicsRequest = new DeleteTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - RequestCompletionHandler callback = new RequestCompletionHandler() { - public void onComplete(ClientResponse response) { - // Do nothing! - } - }; - ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS, callback); - CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); - if (createTopicsResponse.errors().get(topic).code() > 0) { - throw new StreamsException("Could not delete topic: " + topic); + ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS, null); + DeleteTopicsResponse deleteTopicsResponse = new DeleteTopicsResponse(clientResponse.responseBody()); + if (deleteTopicsResponse.errors().get(topic).code() > 0) { + throw new StreamsException("Could not delete topic: " + topic + ". " + deleteTopicsResponse.errors().get(topic).name()); } } @@ -196,37 +163,38 @@ public void onComplete(ClientResponse response) { * @param apiKeys * @param callback */ - public ClientResponse sendRequest(Struct request, ApiKeys apiKeys, RequestCompletionHandler callback) { + public ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys, final RequestCompletionHandler callback) { + Node brokerNode = kafkaClient.leastLoadedNode(new SystemTime().milliseconds()); String brokerId = Integer.toString(brokerNode.id()); + SystemTime systemTime = new SystemTime(); + RequestSend send = new RequestSend(brokerId, kafkaClient.nextRequestHeader(apiKeys), request); - ClientRequest clientRequest = new ClientRequest(new SystemTime().milliseconds(), true, send, callback); - SystemTime systemTime = new SystemTime(); - int iterationCount = 0; - while (iterationCount < maxIterations) { - if (kafkaClient.ready(brokerNode, systemTime.milliseconds())) { - kafkaClient.send(clientRequest, systemTime.milliseconds()); - break; - } else { - kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); - } - iterationCount++; + ClientRequest clientRequest = new ClientRequest(systemTime.milliseconds(), true, send, callback); + + + final long timeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; + while (!kafkaClient.ready(brokerNode, systemTime.milliseconds()) && systemTime.milliseconds() < timeout) { + kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); + } + if (!kafkaClient.ready(brokerNode, systemTime.milliseconds())) { + throw new StreamsException("Request timeout."); } + kafkaClient.send(clientRequest, systemTime.milliseconds()); - iterationCount = 0; + final long timeout2 = systemTime.milliseconds() + MAX_WAIT_TIME_MS; // Poll for the response. - while (iterationCount < maxIterations) { - List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), new SystemTime().milliseconds()); + while (systemTime.milliseconds() < timeout2) { + List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); for (ClientResponse clientResponse: responseList) { if (clientResponse.request().equals(clientRequest)) { return clientResponse; } } - iterationCount++; } throw new StreamsException("StreamsKafkaClient failed to send the request: " + apiKeys.name()); } @@ -237,7 +205,7 @@ public ClientResponse sendRequest(Struct request, ApiKeys apiKeys, RequestComple * @param topic * @return */ - public MetadataResponse.TopicMetadata getTopicMetadata(String topic) { + public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { MetadataRequest metadataRequest = new MetadataRequest(Arrays.asList(topic)); RequestCompletionHandler callback = new RequestCompletionHandler() { @@ -267,7 +235,6 @@ public void onComplete(ClientResponse response) { * @return */ public boolean topicExists(String topicName) { - Map> topics; Properties props = new Properties(); props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, streamsConfig.getList(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG)); @@ -275,7 +242,7 @@ public boolean topicExists(String topicName) { props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); KafkaConsumer consumer = new KafkaConsumer(props); - topics = consumer.listTopics(); + Map> topics = consumer.listTopics(); for (String topicNameInList:topics.keySet()) { if (topicNameInList.equalsIgnoreCase(topicName)) { return true; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java index 4779cdfc0f89..13c71820035b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java @@ -37,9 +37,7 @@ import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueMapper; -import org.apache.kafka.streams.processor.internals.InternalTopicConfig; import org.apache.kafka.streams.processor.internals.ProcessorStateManager; -import org.apache.kafka.streams.processor.internals.StreamsKafkaClient; import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.TestUtils; import org.junit.Before; @@ -51,7 +49,6 @@ import scala.collection.Map; import java.util.Properties; -import java.util.Collections; import java.util.Arrays; import java.util.Locale; import java.util.List; @@ -92,40 +89,6 @@ public void before() { streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); } - //@Test - public void testCreateTopic() throws Exception { - - StreamsConfig streamsConfig = new StreamsConfig(streamsConfiguration); - StreamsKafkaClient streamsKafkaClient = new StreamsKafkaClient(streamsConfig); - - String topicName = "testTopic" + mockTime.milliseconds(); - InternalTopicConfig topicConfig = new InternalTopicConfig(topicName, - Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), - Collections.emptyMap()); - - streamsKafkaClient.createTopic(topicConfig, 1, 1, 0); - - assertTrue(streamsKafkaClient.topicExists(topicName)); - - } - - //@Test - public void testDeleteTopic() throws Exception { - - StreamsConfig streamsConfig = new StreamsConfig(streamsConfiguration); - StreamsKafkaClient streamsKafkaClient = new StreamsKafkaClient(streamsConfig); - - String topicName = "testTopic" + mockTime.milliseconds(); - InternalTopicConfig topicConfig = new InternalTopicConfig(topicName, - Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), - Collections.emptyMap()); - - streamsKafkaClient.createTopic(topicConfig, 1, 1, 0); - streamsKafkaClient.deleteTopic(topicName); - - assertTrue(!streamsKafkaClient.topicExists(topicName)); - } - private Properties getTopicConfigProperties(final String changelog) { // Note: You must initialize the ZkClient with ZKStringSerializer. If you don't, then // createTopic() will only seem to work (it will return without error). The topic will exist in From cb6891fe839cf6ca7ba46ca42437344764cf009e Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Thu, 22 Sep 2016 10:13:01 -0700 Subject: [PATCH 03/16] Made more changes based on review feedback. --- .../apache/kafka/streams/StreamsConfig.java | 2 +- .../internals/InternalTopicManager.java | 34 ++---- .../internals/StreamPartitionAssignor.java | 2 +- .../internals/StreamsKafkaClient.java | 108 ++++++------------ 4 files changed, 48 insertions(+), 98 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 9761cef0eb50..fd64519689a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -286,7 +286,7 @@ public class StreamsConfig extends AbstractConfig { .define(SECURITY_PROTOCOL_CONFIG, Type.STRING, DEFAULT_SECURITY_PROTOCOL, - Importance.LOW, //TODO: Is this the right value? Investigate! + Importance.MEDIUM, SECURITY_PROTOCOL_DOC) .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, ConfigDef.Type.LONG, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index d245a161070d..d2756380fc02 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -18,14 +18,10 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.requests.MetadataResponse; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Locale; -import java.util.Set; import java.util.concurrent.TimeUnit; public class InternalTopicManager { @@ -34,7 +30,6 @@ public class InternalTopicManager { // TODO: the following LogConfig dependency should be removed after KIP-4 public static final String CLEANUP_POLICY_PROP = "cleanup.policy"; - private static final Set CLEANUP_POLICIES = Utils.mkSet("compact", "delete"); public static final String RETENTION_MS = "retention.ms"; public static final Long WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS); @@ -42,29 +37,15 @@ public class InternalTopicManager { private final int replicationFactor; private StreamsKafkaClient streamsKafkaClient; - private StreamsConfig config; - public static boolean isValidCleanupPolicy(final String cleanupPolicy) { - if (cleanupPolicy == null) { - return false; - } - final String[] policies = cleanupPolicy.toLowerCase(Locale.ROOT).split(","); - for (String policy : policies) { - if (!CLEANUP_POLICIES.contains(policy.trim())) { - return false; - } - } - return true; - } public InternalTopicManager() { this.replicationFactor = 0; this.windowChangeLogAdditionalRetention = WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT; } - - public InternalTopicManager(StreamsConfig config, final int replicationFactor, long windowChangeLogAdditionalRetention) { - this.config = config; - this.streamsKafkaClient = new StreamsKafkaClient(config); + + public InternalTopicManager(StreamsKafkaClient streamsKafkaClient, final int replicationFactor, long windowChangeLogAdditionalRetention) { + this.streamsKafkaClient = streamsKafkaClient; this.replicationFactor = replicationFactor; this.windowChangeLogAdditionalRetention = windowChangeLogAdditionalRetention; } @@ -75,18 +56,21 @@ public InternalTopicManager(StreamsConfig config, final int replicationFactor, l * @param topic * @param numPartitions */ - public void makeReady(InternalTopicConfig topic, int numPartitions) { + public void makeReady(final InternalTopicConfig topic, final int numPartitions) { if (!streamsKafkaClient.topicExists(topic.name())) { streamsKafkaClient.createTopic(topic, numPartitions, replicationFactor, windowChangeLogAdditionalRetention); } else { - MetadataResponse.TopicMetadata topicMetadata = streamsKafkaClient.getTopicMetadata(topic.name()); + final MetadataResponse.TopicMetadata topicMetadata = streamsKafkaClient.getTopicMetadata(topic.name()); if (topicMetadata != null) { + if (topicMetadata.error().code() != 0) { + throw new StreamsException("Topic metadata request returned with error code " + topicMetadata.error().code()); + } if (topicMetadata.partitionMetadata().size() != numPartitions) { throw new StreamsException("Topic already exists but the number of partitions is not the same as the requested " + numPartitions + " partitions."); } } else { - throw new StreamsException("Topic metadata is corrupted."); + throw new StreamsException("Could not fetch the topic metadata."); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 42d5df076182..683567b34b27 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -148,7 +148,7 @@ public void configure(Map configs) { } internalTopicManager = new InternalTopicManager( - this.streamThread.config, + new StreamsKafkaClient(this.streamThread.config), configs.containsKey(StreamsConfig.REPLICATION_FACTOR_CONFIG) ? (Integer) configs.get(StreamsConfig.REPLICATION_FACTOR_CONFIG) : 1, configs.containsKey(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) ? (Long) configs.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index b2e626ab6381..5f56b59b6b82 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.Metadata; -import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.ClientRequest; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -37,11 +36,10 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.requests.CreateTopicsResponse; -import org.apache.kafka.common.requests.DeleteTopicsRequest; -import org.apache.kafka.common.requests.DeleteTopicsResponse; import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; @@ -53,9 +51,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.HashMap; -import java.util.HashSet; import java.util.Arrays; import java.util.Properties; import java.util.concurrent.TimeUnit; @@ -64,7 +60,6 @@ public class StreamsKafkaClient { private static final Logger log = LoggerFactory.getLogger(StreamsKafkaClient.class); - private Metrics metrics; private KafkaClient kafkaClient; private StreamsConfig streamsConfig; @@ -77,25 +72,25 @@ public StreamsKafkaClient(final StreamsConfig streamsConfig) { this.streamsConfig = streamsConfig; final Time time = new SystemTime(); - Map metricTags = new LinkedHashMap<>(); + final Map metricTags = new LinkedHashMap<>(); metricTags.put("client-id", StreamsConfig.CLIENT_ID_CONFIG); - Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); - List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); + final Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); + final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); - MetricConfig metricConfig = new MetricConfig().samples(streamsConfig.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG)) + final MetricConfig metricConfig = new MetricConfig().samples(streamsConfig.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(streamsConfig.getLong(CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS) .tags(metricTags); - List reporters = streamsConfig.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, + final List reporters = streamsConfig.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); // TODO: This should come from the KafkaStream reporters.add(new JmxReporter("kafka.streams")); - this.metrics = new Metrics(metricConfig, reporters, time); + final Metrics metrics = new Metrics(metricConfig, reporters, time); - ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(streamsConfig.values()); + final ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(streamsConfig.values()); - Selector selector = new Selector(streamsConfig.getLong(StreamsConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "kafka-client", channelBuilder); + final Selector selector = new Selector(streamsConfig.getLong(StreamsConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, "kafka-client", channelBuilder); kafkaClient = new NetworkClient( selector, @@ -118,77 +113,57 @@ public StreamsKafkaClient(final StreamsConfig streamsConfig) { * @param replicationFactor */ public void createTopic(final InternalTopicConfig internalTopicConfig, final int numPartitions, final int replicationFactor, final long windowChangeLogAdditionalRetention) { - Properties topicProperties = internalTopicConfig.toProperties(windowChangeLogAdditionalRetention); - Map topicConfig = new HashMap<>(); + final Properties topicProperties = internalTopicConfig.toProperties(windowChangeLogAdditionalRetention); + final Map topicConfig = new HashMap<>(); for (String key : topicProperties.stringPropertyNames()) { topicConfig.put(key, topicProperties.getProperty(key)); } - CreateTopicsRequest.TopicDetails topicDetails = new CreateTopicsRequest.TopicDetails(numPartitions, (short) replicationFactor, topicConfig); - Map topics = new HashMap<>(); + final CreateTopicsRequest.TopicDetails topicDetails = new CreateTopicsRequest.TopicDetails(numPartitions, (short) replicationFactor, topicConfig); + final Map topics = new HashMap<>(); topics.put(internalTopicConfig.name(), topicDetails); CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS, null); + ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS); CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); if (createTopicsResponse.errors().get(internalTopicConfig.name()).code() > 0) { throw new StreamsException("Could not create topic: " + internalTopicConfig.name() + ". " + createTopicsResponse.errors().get(internalTopicConfig.name()).name()); } } - /** - * Delete a topic. - * - * @param topic - */ - public void deleteTopic(final String topic) { - log.debug("Deleting topic {} from ZK in partition assignor.", topic); - Set topics = new HashSet(); - topics.add(topic); - - DeleteTopicsRequest deleteTopicsRequest = new DeleteTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - - ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS, null); - DeleteTopicsResponse deleteTopicsResponse = new DeleteTopicsResponse(clientResponse.responseBody()); - if (deleteTopicsResponse.errors().get(topic).code() > 0) { - throw new StreamsException("Could not delete topic: " + topic + ". " + deleteTopicsResponse.errors().get(topic).name()); - } - - } /** * Send a request to kafka broker of this client. Polls the request for a given number of iterations to receive the response. * * @param request * @param apiKeys - * @param callback */ - public ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys, final RequestCompletionHandler callback) { + private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) { - Node brokerNode = kafkaClient.leastLoadedNode(new SystemTime().milliseconds()); - String brokerId = Integer.toString(brokerNode.id()); + final Node brokerNode = kafkaClient.leastLoadedNode(new SystemTime().milliseconds()); + final String brokerId = Integer.toString(brokerNode.id()); - SystemTime systemTime = new SystemTime(); + final SystemTime systemTime = new SystemTime(); RequestSend send = new RequestSend(brokerId, kafkaClient.nextRequestHeader(apiKeys), request); - ClientRequest clientRequest = new ClientRequest(systemTime.milliseconds(), true, send, callback); + final ClientRequest clientRequest = new ClientRequest(systemTime.milliseconds(), true, send, null); - final long timeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; - while (!kafkaClient.ready(brokerNode, systemTime.milliseconds()) && systemTime.milliseconds() < timeout) { + final long readyTimeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; + while (!kafkaClient.ready(brokerNode, systemTime.milliseconds()) && systemTime.milliseconds() < readyTimeout) { kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); } if (!kafkaClient.ready(brokerNode, systemTime.milliseconds())) { - throw new StreamsException("Request timeout."); + throw new StreamsException("Timed out waiting for node=" + brokerNode + " to become available"); } kafkaClient.send(clientRequest, systemTime.milliseconds()); - final long timeout2 = systemTime.milliseconds() + MAX_WAIT_TIME_MS; + final long responseTimeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; // Poll for the response. - while (systemTime.milliseconds() < timeout2) { + while (systemTime.milliseconds() < responseTimeout) { List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); for (ClientResponse clientResponse: responseList) { if (clientResponse.request().equals(clientRequest)) { @@ -196,7 +171,7 @@ public ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys, f } } } - throw new StreamsException("StreamsKafkaClient failed to send the request: " + apiKeys.name()); + throw new StreamsException("Failed to get response from node=" + brokerNode + " within timeout"); } @@ -206,27 +181,18 @@ public ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys, f * @return */ public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { - MetadataRequest metadataRequest = new MetadataRequest(Arrays.asList(topic)); + final MetadataRequest metadataRequest = new MetadataRequest(Arrays.asList(topic)); - RequestCompletionHandler callback = new RequestCompletionHandler() { - public void onComplete(ClientResponse response) { - // Do nothing! - } - }; - - ClientResponse clientResponse = sendRequest(metadataRequest.toStruct(), ApiKeys.METADATA, callback); - MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); + final ClientResponse clientResponse = sendRequest(metadataRequest.toStruct(), ApiKeys.METADATA); + final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { if (topicMetadata.topic().equalsIgnoreCase(topic)) { - if (topicMetadata.error().code() == 0) { - return topicMetadata; - } - return null; + return topicMetadata; } } - return null; + throw new StreamsException("Topic " + topic + " was not found."); } /** @@ -234,16 +200,16 @@ public void onComplete(ClientResponse response) { * @param topicName * @return */ - public boolean topicExists(String topicName) { + public boolean topicExists(final String topicName) { - Properties props = new Properties(); + final Properties props = new Properties(); props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, streamsConfig.getList(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG)); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serdes.String().deserializer().getClass()); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serdes.String().deserializer().getClass()); - KafkaConsumer consumer = new KafkaConsumer(props); - Map> topics = consumer.listTopics(); - for (String topicNameInList:topics.keySet()) { + final KafkaConsumer consumer = new KafkaConsumer<>(props); + final Map> topics = consumer.listTopics(); + for (String topicNameInList : topics.keySet()) { if (topicNameInList.equalsIgnoreCase(topicName)) { return true; } From 393b909158aadada58a796402e8b6f03c0077ef9 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Fri, 23 Sep 2016 10:34:46 -0700 Subject: [PATCH 04/16] Applied review feedback. --- .../internals/InternalTopicManager.java | 16 +++------------ .../internals/StreamsKafkaClient.java | 20 ++++++++----------- .../StreamPartitionAssignorTest.java | 8 ++++---- 3 files changed, 15 insertions(+), 29 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index d2756380fc02..f58ebfbb6040 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -19,16 +19,12 @@ import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.streams.errors.StreamsException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + import java.util.concurrent.TimeUnit; public class InternalTopicManager { - private static final Logger log = LoggerFactory.getLogger(InternalTopicManager.class); - - // TODO: the following LogConfig dependency should be removed after KIP-4 public static final String CLEANUP_POLICY_PROP = "cleanup.policy"; public static final String RETENTION_MS = "retention.ms"; public static final Long WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS); @@ -38,13 +34,7 @@ public class InternalTopicManager { private final int replicationFactor; private StreamsKafkaClient streamsKafkaClient; - - public InternalTopicManager() { - this.replicationFactor = 0; - this.windowChangeLogAdditionalRetention = WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT; - } - - public InternalTopicManager(StreamsKafkaClient streamsKafkaClient, final int replicationFactor, long windowChangeLogAdditionalRetention) { + public InternalTopicManager(final StreamsKafkaClient streamsKafkaClient, final int replicationFactor, final long windowChangeLogAdditionalRetention) { this.streamsKafkaClient = streamsKafkaClient; this.replicationFactor = replicationFactor; this.windowChangeLogAdditionalRetention = windowChangeLogAdditionalRetention; @@ -64,7 +54,7 @@ public void makeReady(final InternalTopicConfig topic, final int numPartitions) final MetadataResponse.TopicMetadata topicMetadata = streamsKafkaClient.getTopicMetadata(topic.name()); if (topicMetadata != null) { if (topicMetadata.error().code() != 0) { - throw new StreamsException("Topic metadata request returned with error code " + topicMetadata.error().code()); + throw new StreamsException("Topic metadata request returned with error code " + topicMetadata.error().code() + ": " + topicMetadata.error().message()); } if (topicMetadata.partitionMetadata().size() != numPartitions) { throw new StreamsException("Topic already exists but the number of partitions is not the same as the requested " + numPartitions + " partitions."); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 5f56b59b6b82..abafec2733cf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -44,25 +44,21 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.HashMap; -import java.util.Arrays; +import java.util.Collections; import java.util.Properties; import java.util.concurrent.TimeUnit; public class StreamsKafkaClient { - private static final Logger log = LoggerFactory.getLogger(StreamsKafkaClient.class); - - private KafkaClient kafkaClient; - private StreamsConfig streamsConfig; + private final KafkaClient kafkaClient; + private final StreamsConfig streamsConfig; private static final int MAX_INFLIGHT_REQUESTS = 100; private static final long MAX_WAIT_TIME_MS = 30000; @@ -122,10 +118,10 @@ public void createTopic(final InternalTopicConfig internalTopicConfig, final int final Map topics = new HashMap<>(); topics.put(internalTopicConfig.name(), topicDetails); - CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); + final CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS); - CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); + final ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS); + final CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); if (createTopicsResponse.errors().get(internalTopicConfig.name()).code() > 0) { throw new StreamsException("Could not create topic: " + internalTopicConfig.name() + ". " + createTopicsResponse.errors().get(internalTopicConfig.name()).name()); } @@ -145,7 +141,7 @@ private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) final SystemTime systemTime = new SystemTime(); - RequestSend send = new RequestSend(brokerId, + final RequestSend send = new RequestSend(brokerId, kafkaClient.nextRequestHeader(apiKeys), request); @@ -181,7 +177,7 @@ private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) * @return */ public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { - final MetadataRequest metadataRequest = new MetadataRequest(Arrays.asList(topic)); + final MetadataRequest metadataRequest = new MetadataRequest(Collections.singletonList(topic)); final ClientResponse clientResponse = sendRequest(metadataRequest.toStruct(), ApiKeys.METADATA); final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index e46a016447db..e2c6c9786063 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -505,7 +505,7 @@ public void testAssignWithInternalTopics() throws Exception { StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, applicationId, client1)); - MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(clientSupplier.restoreConsumer); + MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(thread10.config, clientSupplier.restoreConsumer); partitionAssignor.setInternalTopicManager(internalTopicManager); Map subscriptions = new HashMap<>(); @@ -547,7 +547,7 @@ public void testAssignWithInternalTopicThatsSourceIsAnotherInternalTopic() throw StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, applicationId, client1)); - MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(clientSupplier.restoreConsumer); + MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(thread10.config, clientSupplier.restoreConsumer); partitionAssignor.setInternalTopicManager(internalTopicManager); Map subscriptions = new HashMap<>(); @@ -731,8 +731,8 @@ private class MockInternalTopicManager extends InternalTopicManager { public Map readyTopics = new HashMap<>(); public MockConsumer restoreConsumer; - public MockInternalTopicManager(MockConsumer restoreConsumer) { - super(); + public MockInternalTopicManager(StreamsConfig streamsConfig, MockConsumer restoreConsumer) { + super(new StreamsKafkaClient(streamsConfig), 0, 0); this.restoreConsumer = restoreConsumer; } From 96264d47643b04d53293b86f3ffd718fbb688609 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Wed, 28 Sep 2016 15:15:06 -0700 Subject: [PATCH 05/16] Made changes based on Guozhang's comments. --- build.gradle | 8 --- .../examples/pageview/PageViewTypedDemo.java | 1 - .../pageview/PageViewUntypedDemo.java | 1 - .../examples/wordcount/WordCountDemo.java | 1 - .../wordcount/WordCountProcessorDemo.java | 1 - .../apache/kafka/streams/StreamsConfig.java | 37 +++------- .../internals/InternalTopicManager.java | 2 +- .../internals/StreamPartitionAssignor.java | 4 +- .../internals/StreamsKafkaClient.java | 67 ++++++++++--------- .../integration/FanoutIntegrationTest.java | 1 - ...StreamAggregationDedupIntegrationTest.java | 1 - .../KStreamAggregationIntegrationTest.java | 1 - .../KStreamKTableJoinIntegrationTest.java | 1 - .../KStreamRepartitionJoinTest.java | 1 - .../QueryableStateIntegrationTest.java | 1 - .../integration/ResetIntegrationTest.java | 1 - .../kafka/streams/perf/SimpleBenchmark.java | 4 -- .../streams/smoketest/SmokeTestClient.java | 1 - 18 files changed, 49 insertions(+), 85 deletions(-) diff --git a/build.gradle b/build.gradle index 682f321d17a8..66f0d475bfcd 100644 --- a/build.gradle +++ b/build.gradle @@ -685,14 +685,6 @@ project(':streams') { compile project(':connect:json') // this dependency should be removed after we unify data API compile libs.slf4jApi compile libs.rocksDBJni - // this dependency should be removed after KIP-4 - testCompile (libs.zkclient) { - // users should be able to choose the logging implementation (and slf4j bridge) - exclude module: 'slf4j-log4j12' - exclude module: 'log4j' - exclude module: 'jline' - exclude module: 'netty' - } // zkclient has a hard log4j dependency, we set it here as well to avoid copying an older // version to the binary package; the following two dependencies should be removed after KIP-4 diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index f88f62f0dc3e..47567223862e 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -83,7 +83,6 @@ public static void main(String[] args) throws Exception { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-typed"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index 77cf0ca81ac5..67e8cc50c7df 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -59,7 +59,6 @@ public static void main(String[] args) throws Exception { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-untyped"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index bf1d8cb619ec..99fbc1574b55 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -49,7 +49,6 @@ public static void main(String[] args) throws Exception { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index 1ee6928e98f2..e0560a9c6fd4 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -106,7 +106,6 @@ public static void main(String[] args) throws Exception { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-processor"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index e075036adca0..2d3a638906a8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -58,10 +58,6 @@ public class StreamsConfig extends AbstractConfig { public static final String STATE_DIR_CONFIG = "state.dir"; private static final String STATE_DIR_DOC = "Directory location for state store."; - /** zookeeper.connect */ - public static final String ZOOKEEPER_CONNECT_CONFIG = "zookeeper.connect"; - private static final String ZOOKEEPER_CONNECT_DOC = "Zookeeper connect string for Kafka topics management."; - /** commit.interval.ms */ public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms"; private static final String COMMIT_INTERVAL_MS_DOC = "The frequency with which to save the position of the processor."; @@ -142,29 +138,21 @@ public class StreamsConfig extends AbstractConfig { public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads"; public static final String SECURITY_PROTOCOL_CONFIG = CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; - public static final String SECURITY_PROTOCOL_DOC = CommonClientConfigs.SECURITY_PROTOCOL_DOC; public static final String DEFAULT_SECURITY_PROTOCOL = CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL; public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; - public static final String CONNECTIONS_MAX_IDLE_MS_DOC = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC; public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; - public static final String RETRY_BACKOFF_MS_DOC = CommonClientConfigs.RETRY_BACKOFF_MS_DOC; public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; - public static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; - public static final String RECONNECT_BACKOFF_MS_DOC = CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC; public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; - public static final String SEND_BUFFER_DOC = CommonClientConfigs.SEND_BUFFER_DOC; public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; - public static final String RECEIVE_BUFFER_DOC = CommonClientConfigs.RECEIVE_BUFFER_DOC; public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; - public static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; static { @@ -181,11 +169,6 @@ public class StreamsConfig extends AbstractConfig { "", Importance.HIGH, CommonClientConfigs.CLIENT_ID_DOC) - .define(ZOOKEEPER_CONNECT_CONFIG, - Type.STRING, - "", - Importance.HIGH, - StreamsConfig.ZOOKEEPER_CONNECT_DOC) .define(STATE_DIR_CONFIG, Type.STRING, "/tmp/kafka-streams", @@ -288,48 +271,48 @@ public class StreamsConfig extends AbstractConfig { Type.STRING, DEFAULT_SECURITY_PROTOCOL, Importance.MEDIUM, - SECURITY_PROTOCOL_DOC) + CommonClientConfigs.SECURITY_PROTOCOL_DOC) .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, ConfigDef.Type.LONG, 9 * 60 * 1000, ConfigDef.Importance.MEDIUM, - CONNECTIONS_MAX_IDLE_MS_DOC) + CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) .define(RETRY_BACKOFF_MS_CONFIG, ConfigDef.Type.LONG, 100L, atLeast(0L), ConfigDef.Importance.LOW, - RETRY_BACKOFF_MS_DOC) + CommonClientConfigs.RETRY_BACKOFF_MS_DOC) .define(METADATA_MAX_AGE_CONFIG, ConfigDef.Type.LONG, 5 * 60 * 1000, atLeast(0), ConfigDef.Importance.LOW, - METADATA_MAX_AGE_DOC) + CommonClientConfigs.METADATA_MAX_AGE_DOC) .define(RECONNECT_BACKOFF_MS_CONFIG, ConfigDef.Type.LONG, 50L, atLeast(0L), ConfigDef.Importance.LOW, - RECONNECT_BACKOFF_MS_DOC) + CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC) .define(SEND_BUFFER_CONFIG, ConfigDef.Type.INT, - 128 * 1024, + 32 * 1024, atLeast(0), ConfigDef.Importance.MEDIUM, - SEND_BUFFER_DOC) + CommonClientConfigs.SEND_BUFFER_DOC) .define(RECEIVE_BUFFER_CONFIG, ConfigDef.Type.INT, 32 * 1024, atLeast(0), ConfigDef.Importance.MEDIUM, - RECEIVE_BUFFER_DOC) + CommonClientConfigs.RECEIVE_BUFFER_DOC) .define(REQUEST_TIMEOUT_MS_CONFIG, ConfigDef.Type.INT, 40 * 1000, atLeast(0), ConfigDef.Importance.MEDIUM, - REQUEST_TIMEOUT_MS_DOC); + CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC); } // this is the list of configs for underlying clients @@ -417,8 +400,6 @@ public Map getConsumerConfigs(StreamThread streamThread, String props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, getInt(NUM_STANDBY_REPLICAS_CONFIG)); props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, StreamPartitionAssignor.class.getName()); props.put(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, getLong(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG)); - if (!getString(ZOOKEEPER_CONNECT_CONFIG).equals("")) - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, getString(ZOOKEEPER_CONNECT_CONFIG)); props.put(APPLICATION_SERVER_CONFIG, getString(APPLICATION_SERVER_CONFIG)); return props; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index f58ebfbb6040..aa798666f92f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -32,7 +32,7 @@ public class InternalTopicManager { private final long windowChangeLogAdditionalRetention; private final int replicationFactor; - private StreamsKafkaClient streamsKafkaClient; + private final StreamsKafkaClient streamsKafkaClient; public InternalTopicManager(final StreamsKafkaClient streamsKafkaClient, final int replicationFactor, final long windowChangeLogAdditionalRetention) { this.streamsKafkaClient = streamsKafkaClient; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 683567b34b27..2e2b83270740 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -237,8 +237,8 @@ private Map prepareTopic(Map reporters = streamsConfig.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); // TODO: This should come from the KafkaStream - reporters.add(new JmxReporter("kafka.streams")); + reporters.add(new JmxReporter("kafka.admin")); final Metrics metrics = new Metrics(metricConfig, reporters, time); final ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(streamsConfig.values()); @@ -97,7 +94,6 @@ public StreamsKafkaClient(final StreamsConfig streamsConfig) { streamsConfig.getInt(StreamsConfig.SEND_BUFFER_CONFIG), streamsConfig.getInt(StreamsConfig.RECEIVE_BUFFER_CONFIG), streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG), time); - } @@ -129,45 +125,61 @@ public void createTopic(final InternalTopicConfig internalTopicConfig, final int /** - * Send a request to kafka broker of this client. Polls the request for a given number of iterations to receive the response. + * Send a request to kafka broker of this client. Keep polling until the corresponding response is received. * * @param request * @param apiKeys */ private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) { - final Node brokerNode = kafkaClient.leastLoadedNode(new SystemTime().milliseconds()); - final String brokerId = Integer.toString(brokerNode.id()); - + String brokerId = null; final SystemTime systemTime = new SystemTime(); + final Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); + final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); + metadata.update(Cluster.bootstrap(addresses), systemTime.milliseconds()); + + final List nodes = metadata.fetch().nodes(); + final long readyTimeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; + boolean foundNode = false; + while (!foundNode && (systemTime.milliseconds() < readyTimeout)) { + for (Node node: nodes) { + if (kafkaClient.ready(node, systemTime.milliseconds())) { + brokerId = Integer.toString(node.id()); + foundNode = true; + break; + } + } + kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); + } + if (brokerId == null) { + throw new StreamsException("Could not find any available broker."); + } + final RequestSend send = new RequestSend(brokerId, kafkaClient.nextRequestHeader(apiKeys), request); final ClientRequest clientRequest = new ClientRequest(systemTime.milliseconds(), true, send, null); - - final long readyTimeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; - while (!kafkaClient.ready(brokerNode, systemTime.milliseconds()) && systemTime.milliseconds() < readyTimeout) { - kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); - } - if (!kafkaClient.ready(brokerNode, systemTime.milliseconds())) { - throw new StreamsException("Timed out waiting for node=" + brokerNode + " to become available"); - } kafkaClient.send(clientRequest, systemTime.milliseconds()); final long responseTimeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; // Poll for the response. while (systemTime.milliseconds() < responseTimeout) { List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); - for (ClientResponse clientResponse: responseList) { - if (clientResponse.request().equals(clientRequest)) { - return clientResponse; + if (!responseList.isEmpty()) { + if (responseList.size() > 1) { + throw new StreamsException("Sent one request but received multiple or no responses."); + } + if (responseList.get(0).request().equals(clientRequest)) { + return responseList.get(0); + } else { + throw new StreamsException("Inconsistent response received."); } } } - throw new StreamsException("Failed to get response from node=" + brokerNode + " within timeout"); + throw new StreamsException("Failed to get response from broker within timeout"); } @@ -198,15 +210,10 @@ public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { */ public boolean topicExists(final String topicName) { - final Properties props = new Properties(); - props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, streamsConfig.getList(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG)); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Serdes.String().deserializer().getClass()); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Serdes.String().deserializer().getClass()); - - final KafkaConsumer consumer = new KafkaConsumer<>(props); - final Map> topics = consumer.listTopics(); - for (String topicNameInList : topics.keySet()) { - if (topicNameInList.equalsIgnoreCase(topicName)) { + final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); + final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); + for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { + if (topicMetadata.topic().equalsIgnoreCase(topicName)) { return true; } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/FanoutIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/FanoutIntegrationTest.java index a5fb07631870..f40569537fa4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/FanoutIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/FanoutIntegrationTest.java @@ -113,7 +113,6 @@ public void shouldFanoutTheInput() throws Exception { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "fanout-integration-test"); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSizeBytes); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java index ab08dbe0f596..bbdef6fce59a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java @@ -78,7 +78,6 @@ public void before() { streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration .put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 2000); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java index e5560c1b62fc..44ec4c0f84b2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java @@ -99,7 +99,6 @@ public void before() { streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration .put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java index 02beee3d39ce..a38adec90b9b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java @@ -86,7 +86,6 @@ public void before() { streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "join-integration-test-" + testNo); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java index 2ac9c4714b0e..0ac943c0bdc4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java @@ -99,7 +99,6 @@ public void before() { streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index 66b6d2e971c9..3bc695560e9b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -136,7 +136,6 @@ public void before() throws IOException { streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration .put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory("qs-test").getPath()); streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java index 0f1717c11226..b966428da8cc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java @@ -163,7 +163,6 @@ private Properties prepareTest() throws Exception { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java index 630167a10f4a..264be3be4ba1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java @@ -142,7 +142,6 @@ private Properties setJoinProperties(final String applicationId) { props.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); @@ -425,7 +424,6 @@ private KafkaStreams createKafkaStreams(String topic, File stateDir, String kafk props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams"); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); @@ -468,7 +466,6 @@ private KafkaStreams createKafkaStreamsWithSink(String topic, File stateDir, Str props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams-with-sink"); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); @@ -560,7 +557,6 @@ private KafkaStreams createKafkaStreamsWithStateStore(String topic, File stateDi props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams-with-store"); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java index f920c515c0e1..e326188b9c9b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java @@ -83,7 +83,6 @@ private static KafkaStreams createKafkaStreams(File stateDir, String kafka, Stri props.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TestTimestampExtractor.class.getName()); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); From 5e386f0fe3c109dfea95b392569adbe8f32018a0 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Thu, 29 Sep 2016 17:22:15 -0700 Subject: [PATCH 06/16] Disabled auto topic generation for EmbeddedKafkaCluster. Removed polling in StreamPartitionAssignor. --- .../internals/InternalTopicManager.java | 6 +++++- .../internals/StreamPartitionAssignor.java | 7 +------ .../processor/internals/StreamsKafkaClient.java | 16 ++++------------ .../KStreamAggregationIntegrationTest.java | 1 + .../integration/utils/EmbeddedKafkaCluster.java | 1 + 5 files changed, 12 insertions(+), 19 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index aa798666f92f..f187a11df40c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -50,6 +50,10 @@ public void makeReady(final InternalTopicConfig topic, final int numPartitions) if (!streamsKafkaClient.topicExists(topic.name())) { streamsKafkaClient.createTopic(topic, numPartitions, replicationFactor, windowChangeLogAdditionalRetention); + // Make sure the topic was created. + if (!streamsKafkaClient.topicExists(topic.name())) { + throw new StreamsException("Cound not create topic: " + topic.name()); + } } else { final MetadataResponse.TopicMetadata topicMetadata = streamsKafkaClient.getTopicMetadata(topic.name()); if (topicMetadata != null) { @@ -57,7 +61,7 @@ public void makeReady(final InternalTopicConfig topic, final int numPartitions) throw new StreamsException("Topic metadata request returned with error code " + topicMetadata.error().code() + ": " + topicMetadata.error().message()); } if (topicMetadata.partitionMetadata().size() != numPartitions) { - throw new StreamsException("Topic already exists but the number of partitions is not the same as the requested " + numPartitions + " partitions."); + throw new StreamsException("Topic already exists but the number of partitions, " + topicMetadata.partitionMetadata().size() + ", is not the same as the requested " + numPartitions + " partitions."); } } else { throw new StreamsException("Could not fetch the topic metadata."); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 2e2b83270740..f65c3d14c2a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -217,12 +217,7 @@ private Map prepareTopic(Map partitions; - do { - partitions = streamThread.restoreConsumer.partitionsFor(topic.name()); - } while (partitions == null || partitions.size() != numPartitions); - + List partitions = streamThread.restoreConsumer.partitionsFor(topic.name()); for (PartitionInfo partition : partitions) partitionInfos.put(new TopicPartition(partition.topic(), partition.partition()), partition); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 478cf6c406ea..1617e9721b95 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -47,7 +47,6 @@ import java.util.List; import java.util.Map; import java.util.HashMap; -import java.util.Collections; import java.util.Properties; import java.util.concurrent.TimeUnit; @@ -189,18 +188,15 @@ private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) * @return */ public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { - final MetadataRequest metadataRequest = new MetadataRequest(Collections.singletonList(topic)); - final ClientResponse clientResponse = sendRequest(metadataRequest.toStruct(), ApiKeys.METADATA); + final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); - for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { if (topicMetadata.topic().equalsIgnoreCase(topic)) { return topicMetadata; } } - - throw new StreamsException("Topic " + topic + " was not found."); + return null; } /** @@ -210,12 +206,8 @@ public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { */ public boolean topicExists(final String topicName) { - final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); - final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); - for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { - if (topicMetadata.topic().equalsIgnoreCase(topicName)) { - return true; - } + if (getTopicMetadata(topicName) != null) { + return true; } return false; } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java index 44ec4c0f84b2..778df8cf5e2d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java @@ -104,6 +104,7 @@ public void before() { streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1); streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSizeBytes); + final KeyValueMapper mapper = MockKeyValueMapper.SelectValueMapper(); stream = builder.stream(Serdes.Integer(), Serdes.String(), streamOneInput); groupedStream = stream diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java index 9c0cbe1ee7e5..fc28ad5f06e9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java @@ -58,6 +58,7 @@ public void start() throws IOException, InterruptedException { brokerConfig.put(KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); brokerConfig.put(KafkaConfig$.MODULE$.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L); brokerConfig.put(KafkaConfig$.MODULE$.GroupMinSessionTimeoutMsProp(), 0); + brokerConfig.put(KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), false); for (int i = 0; i < brokers.length; i++) { brokerConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), i); From f44a9853cbd38f9c10a0b15bc2297d8a242fce3f Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Fri, 30 Sep 2016 13:22:04 -0700 Subject: [PATCH 07/16] Now deleting and re creating existing topic with different number of partitions. --- .../internals/InternalTopicManager.java | 8 ++++++- .../internals/StreamsKafkaClient.java | 21 ++++++++++++++++++- .../utils/EmbeddedKafkaCluster.java | 1 - 3 files changed, 27 insertions(+), 3 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index f187a11df40c..da3d40dbdc5d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -61,7 +61,13 @@ public void makeReady(final InternalTopicConfig topic, final int numPartitions) throw new StreamsException("Topic metadata request returned with error code " + topicMetadata.error().code() + ": " + topicMetadata.error().message()); } if (topicMetadata.partitionMetadata().size() != numPartitions) { - throw new StreamsException("Topic already exists but the number of partitions, " + topicMetadata.partitionMetadata().size() + ", is not the same as the requested " + numPartitions + " partitions."); + // Delete the topic and create a new one with the requested number of partitions. + streamsKafkaClient.deleteTopic(topic); + streamsKafkaClient.createTopic(topic, numPartitions, replicationFactor, windowChangeLogAdditionalRetention); + if (!streamsKafkaClient.topicExists(topic.name())) { + throw new StreamsException("Deleted the topic " + topic.name() + " but could not create it again."); + } + } } else { throw new StreamsException("Could not fetch the topic metadata."); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 1617e9721b95..241fd26a5fec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -34,6 +34,8 @@ import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.requests.CreateTopicsResponse; +import org.apache.kafka.common.requests.DeleteTopicsRequest; +import org.apache.kafka.common.requests.DeleteTopicsResponse; import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; @@ -49,7 +51,8 @@ import java.util.HashMap; import java.util.Properties; import java.util.concurrent.TimeUnit; - +import java.util.Set; +import java.util.HashSet; public class StreamsKafkaClient { @@ -123,6 +126,22 @@ public void createTopic(final InternalTopicConfig internalTopicConfig, final int } + /** + * Delete a given topic. + * + * @param internalTopicConfig + */ + public void deleteTopic(final InternalTopicConfig internalTopicConfig) { + final Set topics = new HashSet(); + topics.add(internalTopicConfig.name()); + final DeleteTopicsRequest deleteTopicsRequest = new DeleteTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); + final ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS); + final DeleteTopicsResponse deleteTopicsResponse = new DeleteTopicsResponse(clientResponse.responseBody()); + if (deleteTopicsResponse.errors().get(internalTopicConfig.name()).code() > 0) { + throw new StreamsException("Could not delete topic: " + internalTopicConfig.name()); + } + } + /** * Send a request to kafka broker of this client. Keep polling until the corresponding response is received. * diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java index fc28ad5f06e9..9c0cbe1ee7e5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java @@ -58,7 +58,6 @@ public void start() throws IOException, InterruptedException { brokerConfig.put(KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); brokerConfig.put(KafkaConfig$.MODULE$.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L); brokerConfig.put(KafkaConfig$.MODULE$.GroupMinSessionTimeoutMsProp(), 0); - brokerConfig.put(KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), false); for (int i = 0; i < brokers.length; i++) { brokerConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), i); From 3d9dba21f0c39104f5e807a50f3fdb5956eda1bf Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Fri, 30 Sep 2016 15:09:51 -0700 Subject: [PATCH 08/16] Fixed some issues in tests. --- .../processor/internals/InternalTopicManager.java | 5 +++++ .../processor/internals/StreamPartitionAssignor.java | 4 ++++ .../kafka/streams/processor/internals/StreamThread.java | 6 ++++++ .../streams/processor/internals/StreamsKafkaClient.java | 9 ++++++++- .../processor/internals/StreamPartitionAssignorTest.java | 5 ++++- 5 files changed, 27 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index da3d40dbdc5d..85548617c47d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -21,6 +21,7 @@ import org.apache.kafka.streams.errors.StreamsException; +import java.io.IOException; import java.util.concurrent.TimeUnit; public class InternalTopicManager { @@ -75,4 +76,8 @@ public void makeReady(final InternalTopicConfig topic, final int numPartitions) } } + + public void close() throws IOException { + streamsKafkaClient.close(); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index f65c3d14c2a0..e81af00214bf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -39,6 +39,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -662,4 +663,7 @@ public boolean hasUpdates() { } + public void close() throws IOException { + internalTopicManager.close(); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index f1913b882741..a7c99e2002ca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -290,6 +290,12 @@ private void shutdown() { log.error("stream-thread [{}] Failed to close restore consumer: ", this.getName(), e); } + try { + partitionAssignor.close(); + } catch (Throwable e) { + log.error("stream-thread [{}] Failed to close KafkaStreamClient: ", this.getName(), e); + } + removeStreamTasks(); log.info("stream-thread [{}] Stream thread shutdown complete", this.getName()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 241fd26a5fec..8d41735d8fbf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -44,6 +44,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; +import java.io.IOException; import java.net.InetSocketAddress; import java.util.LinkedHashMap; import java.util.List; @@ -57,6 +58,7 @@ public class StreamsKafkaClient { private final KafkaClient kafkaClient; + private final List reporters; private final StreamsConfig streamsConfig; private static final int MAX_INFLIGHT_REQUESTS = 100; @@ -77,7 +79,7 @@ public StreamsKafkaClient(final StreamsConfig streamsConfig) { final MetricConfig metricConfig = new MetricConfig().samples(streamsConfig.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(streamsConfig.getLong(CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS) .tags(metricTags); - final List reporters = streamsConfig.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, + reporters = streamsConfig.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); // TODO: This should come from the KafkaStream reporters.add(new JmxReporter("kafka.admin")); @@ -98,6 +100,11 @@ public StreamsKafkaClient(final StreamsConfig streamsConfig) { streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG), time); } + public void close() throws IOException { + for (MetricsReporter metricsReporter: this.reporters) { + metricsReporter.close(); + } + } /** * Cretes a new topic with the given number of partitions and replication factor. diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index e2c6c9786063..ffd3848dd0c3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -303,10 +303,13 @@ public void testAssignWithStates() throws Exception { String client1 = "client1"; - StreamThread thread10 = new StreamThread(builder, config, new MockClientSupplier(), applicationId, client1, uuid1, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); + MockClientSupplier mockClientSupplier = new MockClientSupplier(); + StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, applicationId, client1, uuid1, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); + partitionAssignor.configure(config.getConsumerConfigs(thread10, applicationId, client1)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); Map subscriptions = new HashMap<>(); subscriptions.put("consumer10", From 5b0be7aa1a2b1d26ae453aab78fb4989c9145411 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Fri, 7 Oct 2016 11:41:42 -0700 Subject: [PATCH 09/16] Sending topic management request in batch. --- .../apache/kafka/streams/StreamsConfig.java | 73 +++++- .../internals/InternalTopicManager.java | 239 ++---------------- .../internals/StreamPartitionAssignor.java | 68 ++--- .../processor/internals/StreamThread.java | 6 + .../StreamPartitionAssignorTest.java | 29 ++- 5 files changed, 154 insertions(+), 261 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 66c15b90a02a..0176aa86c322 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -141,6 +141,31 @@ public class StreamsConfig extends AbstractConfig { public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering"; public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads"; + public static final String SECURITY_PROTOCOL_CONFIG = CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; + public static final String SECURITY_PROTOCOL_DOC = CommonClientConfigs.SECURITY_PROTOCOL_DOC; + public static final String DEFAULT_SECURITY_PROTOCOL = CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL; + + public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; + public static final String CONNECTIONS_MAX_IDLE_MS_DOC = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC; + + public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; + public static final String RETRY_BACKOFF_MS_DOC = CommonClientConfigs.RETRY_BACKOFF_MS_DOC; + + public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; + public static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; + + public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; + public static final String RECONNECT_BACKOFF_MS_DOC = CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC; + + public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; + public static final String SEND_BUFFER_DOC = CommonClientConfigs.SEND_BUFFER_DOC; + + public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; + public static final String RECEIVE_BUFFER_DOC = CommonClientConfigs.RECEIVE_BUFFER_DOC; + + public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; + public static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; + static { CONFIG = new ConfigDef().define(APPLICATION_ID_CONFIG, // required with no default value Type.STRING, @@ -257,7 +282,53 @@ public class StreamsConfig extends AbstractConfig { 10 * 1024 * 1024L, atLeast(0), Importance.LOW, - CACHE_MAX_BYTES_BUFFERING_DOC); + CACHE_MAX_BYTES_BUFFERING_DOC) + .define(SECURITY_PROTOCOL_CONFIG, + Type.STRING, + DEFAULT_SECURITY_PROTOCOL, + Importance.MEDIUM, + SECURITY_PROTOCOL_DOC) + .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, + ConfigDef.Type.LONG, + 9 * 60 * 1000, + ConfigDef.Importance.MEDIUM, + CONNECTIONS_MAX_IDLE_MS_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, + ConfigDef.Type.LONG, + 100L, + atLeast(0L), + ConfigDef.Importance.LOW, + RETRY_BACKOFF_MS_DOC) + .define(METADATA_MAX_AGE_CONFIG, + ConfigDef.Type.LONG, + 5 * 60 * 1000, + atLeast(0), + ConfigDef.Importance.LOW, + METADATA_MAX_AGE_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, + ConfigDef.Type.LONG, + 50L, + atLeast(0L), + ConfigDef.Importance.LOW, + RECONNECT_BACKOFF_MS_DOC) + .define(SEND_BUFFER_CONFIG, + ConfigDef.Type.INT, + 128 * 1024, + atLeast(0), + ConfigDef.Importance.MEDIUM, + SEND_BUFFER_DOC) + .define(RECEIVE_BUFFER_CONFIG, + ConfigDef.Type.INT, + 32 * 1024, + atLeast(0), + ConfigDef.Importance.MEDIUM, + RECEIVE_BUFFER_DOC) + .define(REQUEST_TIMEOUT_MS_CONFIG, + ConfigDef.Type.INT, + 40 * 1000, + atLeast(0), + ConfigDef.Importance.MEDIUM, + REQUEST_TIMEOUT_MS_DOC); } // this is the list of configs for underlying clients diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index a65a2ae35732..63d24d0d0f09 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -17,242 +17,55 @@ package org.apache.kafka.streams.processor.internals; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.I0Itec.zkclient.ZkClient; -import org.I0Itec.zkclient.exception.ZkNoNodeException; -import org.I0Itec.zkclient.exception.ZkNodeExistsException; -import org.I0Itec.zkclient.serialize.ZkSerializer; -import org.apache.kafka.streams.errors.StreamsException; -import org.apache.zookeeper.ZooDefs; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.kafka.common.requests.MetadataResponse; import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.Collections; +import java.util.Collection; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; public class InternalTopicManager { - private static final Logger log = LoggerFactory.getLogger(InternalTopicManager.class); - - // TODO: the following ZK dependency should be removed after KIP-4 - private static final String ZK_TOPIC_PATH = "/brokers/topics"; - private static final String ZK_BROKER_PATH = "/brokers/ids"; - private static final String ZK_DELETE_TOPIC_PATH = "/admin/delete_topics"; - private static final String ZK_ENTITY_CONFIG_PATH = "/config/topics"; - // TODO: the following LogConfig dependency should be removed after KIP-4 public static final String CLEANUP_POLICY_PROP = "cleanup.policy"; public static final String RETENTION_MS = "retention.ms"; public static final Long WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS); - private final ZkClient zkClient; - private final int replicationFactor; private final long windowChangeLogAdditionalRetention; - private class ZKStringSerializer implements ZkSerializer { - - /** - * @throws AssertionError if the byte String encoding type is not supported - */ - @Override - public byte[] serialize(Object data) { - try { - return ((String) data).getBytes("UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new AssertionError(e); - } - } - - /** - * @throws AssertionError if the byte String encoding type is not supported - */ - @Override - public Object deserialize(byte[] bytes) { - try { - if (bytes == null) - return null; - else - return new String(bytes, "UTF-8"); - } catch (UnsupportedEncodingException e) { - throw new AssertionError(e); - } - } - } - - public InternalTopicManager() { - this.zkClient = null; - this.replicationFactor = 0; - this.windowChangeLogAdditionalRetention = WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT; - } + private final int replicationFactor; + private final StreamsKafkaClient streamsKafkaClient; - public InternalTopicManager(String zkConnect, final int replicationFactor, long windowChangeLogAdditionalRetention) { - this.zkClient = new ZkClient(zkConnect, 30 * 1000, 30 * 1000, new ZKStringSerializer()); + public InternalTopicManager(final StreamsKafkaClient streamsKafkaClient, final int replicationFactor, final long windowChangeLogAdditionalRetention) { + this.streamsKafkaClient = streamsKafkaClient; this.replicationFactor = replicationFactor; this.windowChangeLogAdditionalRetention = windowChangeLogAdditionalRetention; } - public void makeReady(InternalTopicConfig topic, int numPartitions) { - boolean topicNotReady = true; - - while (topicNotReady) { - Map> topicMetadata = getTopicMetadata(topic.name()); - - if (topicMetadata == null) { - try { - createTopic(topic, numPartitions, replicationFactor); - } catch (ZkNodeExistsException e) { - // ignore and continue - } - } else { - if (topicMetadata.size() > numPartitions) { - // else if topic exists with more #.partitions than needed, delete in order to re-create it - try { - deleteTopic(topic.name()); - } catch (ZkNodeExistsException e) { - // ignore and continue - } - } else if (topicMetadata.size() < numPartitions) { - // else if topic exists with less #.partitions than needed, add partitions - try { - addPartitions(topic.name(), numPartitions - topicMetadata.size(), replicationFactor, topicMetadata); - } catch (ZkNoNodeException e) { - // ignore and continue - } - } else { - topicNotReady = false; - } - } - } + public void makeReady(final InternalTopicConfig topic, int numPartitions) { + Map topics = new HashMap<>(); + topics.put(topic, numPartitions); + makeReady(topics); } - private List getBrokers() { - List brokers = new ArrayList<>(); - for (String broker: zkClient.getChildren(ZK_BROKER_PATH)) { - brokers.add(Integer.parseInt(broker)); - } - Collections.sort(brokers); - - log.debug("Read brokers {} from ZK in partition assignor.", brokers); - - return brokers; + /** + * Prepares the set of given internal topics. If the topic with the correct number of partitions exists ignores it. For the ones with different number of + * partitions delete them and create new ones with correct number of partitons along with the non existing topics. + * @param topics + */ + public void makeReady(final Map topics) { + + //TODO: Add loop/ inspect th error codes + Collection topicMetadatas = streamsKafkaClient.fetchTopicMetadata(); + Map topicsToBeDeleted = streamsKafkaClient.getTopicsToBeDeleted(topics, topicMetadatas); + Map topicsToBeCreated = streamsKafkaClient.filterExistingTopics(topics, topicMetadatas); + topicsToBeCreated.putAll(topicsToBeDeleted); + streamsKafkaClient.deleteTopics(topicsToBeDeleted); + streamsKafkaClient.createTopics(topicsToBeCreated, replicationFactor, windowChangeLogAdditionalRetention); } - @SuppressWarnings("unchecked") - private Map> getTopicMetadata(String topic) { - String data = zkClient.readData(ZK_TOPIC_PATH + "/" + topic, true); - - if (data == null) return null; - - try { - ObjectMapper mapper = new ObjectMapper(); - - Map dataMap = mapper.readValue(data, new TypeReference>() { - - }); - - Map> partitions = (Map>) dataMap.get("partitions"); - - log.debug("Read partitions {} for topic {} from ZK in partition assignor.", partitions, topic); - - return partitions; - } catch (IOException e) { - throw new StreamsException("Error while reading topic metadata from ZK for internal topic " + topic, e); - } - } - - private void createTopic(InternalTopicConfig topic, int numPartitions, int replicationFactor) throws ZkNodeExistsException { - log.debug("Creating topic {} with {} partitions from ZK in partition assignor.", topic.name(), numPartitions); - ObjectMapper mapper = new ObjectMapper(); - List brokers = getBrokers(); - int numBrokers = brokers.size(); - if (numBrokers < replicationFactor) { - log.warn("Not enough brokers found. The replication factor is reduced from " + replicationFactor + " to " + numBrokers); - replicationFactor = numBrokers; - } - - Map> assignment = new HashMap<>(); - - for (int i = 0; i < numPartitions; i++) { - ArrayList brokerList = new ArrayList<>(); - for (int r = 0; r < replicationFactor; r++) { - int shift = r * numBrokers / replicationFactor; - brokerList.add(brokers.get((i + shift) % numBrokers)); - } - assignment.put(i, brokerList); - } - // write out config first just like in AdminUtils.scala createOrUpdateTopicPartitionAssignmentPathInZK() - try { - Map dataMap = new HashMap<>(); - dataMap.put("version", 1); - dataMap.put("config", topic.toProperties(windowChangeLogAdditionalRetention)); - String data = mapper.writeValueAsString(dataMap); - zkClient.createPersistent(ZK_ENTITY_CONFIG_PATH + "/" + topic.name(), data, ZooDefs.Ids.OPEN_ACL_UNSAFE); - } catch (JsonProcessingException e) { - throw new StreamsException("Error while creating topic config in ZK for internal topic " + topic, e); - } - - // try to write to ZK with open ACL - try { - Map dataMap = new HashMap<>(); - dataMap.put("version", 1); - dataMap.put("partitions", assignment); - String data = mapper.writeValueAsString(dataMap); - - zkClient.createPersistent(ZK_TOPIC_PATH + "/" + topic.name(), data, ZooDefs.Ids.OPEN_ACL_UNSAFE); - } catch (JsonProcessingException e) { - throw new StreamsException("Error while creating topic metadata in ZK for internal topic " + topic, e); - } - } - - private void deleteTopic(String topic) throws ZkNodeExistsException { - log.debug("Deleting topic {} from ZK in partition assignor.", topic); - - zkClient.createPersistent(ZK_DELETE_TOPIC_PATH + "/" + topic, "", ZooDefs.Ids.OPEN_ACL_UNSAFE); - } - - private void addPartitions(String topic, int numPartitions, int replicationFactor, Map> existingAssignment) { - log.debug("Adding {} partitions topic {} from ZK with existing partitions assigned as {} in partition assignor.", topic, numPartitions, existingAssignment); - - List brokers = getBrokers(); - int numBrokers = brokers.size(); - if (numBrokers < replicationFactor) { - log.warn("Not enough brokers found. The replication factor is reduced from " + replicationFactor + " to " + numBrokers); - replicationFactor = numBrokers; - } - - int startIndex = existingAssignment.size(); - - Map> newAssignment = new HashMap<>(existingAssignment); - - for (int i = 0; i < numPartitions; i++) { - ArrayList brokerList = new ArrayList<>(); - for (int r = 0; r < replicationFactor; r++) { - int shift = r * numBrokers / replicationFactor; - brokerList.add(brokers.get((i + shift) % numBrokers)); - } - newAssignment.put(i + startIndex, brokerList); - } - - // try to write to ZK with open ACL - try { - Map dataMap = new HashMap<>(); - dataMap.put("version", 1); - dataMap.put("partitions", newAssignment); - - ObjectMapper mapper = new ObjectMapper(); - String data = mapper.writeValueAsString(dataMap); - - zkClient.writeData(ZK_TOPIC_PATH + "/" + topic, data); - } catch (JsonProcessingException e) { - throw new StreamsException("Error while updating topic metadata in ZK for internal topic " + topic, e); - } + public void close() throws IOException { + streamsKafkaClient.close(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 3be9c114ba7f..1ac4f449a9a8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -39,6 +39,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -133,7 +134,7 @@ public void configure(Map configs) { final String[] hostPort = userEndPoint.split(":"); if (hostPort.length != 2) { throw new ConfigException(String.format("stream-thread [%s] Config %s isn't in the correct format. Expected a host:port pair" + - " but received %s", + " but received %s", streamThread.getName(), StreamsConfig.APPLICATION_SERVER_CONFIG, userEndPoint)); } else { try { @@ -147,16 +148,12 @@ public void configure(Map configs) { } - if (configs.containsKey(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG)) { - internalTopicManager = new InternalTopicManager( - (String) configs.get(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG), - configs.containsKey(StreamsConfig.REPLICATION_FACTOR_CONFIG) ? (Integer) configs.get(StreamsConfig.REPLICATION_FACTOR_CONFIG) : 1, - configs.containsKey(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) ? - (Long) configs.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) - : WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT); - } else { - log.info("stream-thread [{}] Config '{}' isn't supplied and hence no internal topics will be created.", streamThread.getName(), StreamsConfig.ZOOKEEPER_CONNECT_CONFIG); - } + internalTopicManager = new InternalTopicManager( + new StreamsKafkaClient(this.streamThread.config), + configs.containsKey(StreamsConfig.REPLICATION_FACTOR_CONFIG) ? (Integer) configs.get(StreamsConfig.REPLICATION_FACTOR_CONFIG) : 1, + configs.containsKey(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) ? + (Long) configs.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) + : WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT); } @Override @@ -202,6 +199,7 @@ private Map prepareTopic(Map topics = new HashMap<>(); for (Map.Entry> entry : topicToTaskIds.entrySet()) { InternalTopicConfig topic = entry.getKey(); int numPartitions = 0; @@ -218,15 +216,14 @@ private Map prepareTopic(Map partitions; - do { - partitions = streamThread.restoreConsumer.partitionsFor(topic.name()); - } while (partitions == null || partitions.size() != numPartitions); + internalTopicManager.makeReady(topics); + for (Map.Entry> entry : topicToTaskIds.entrySet()) { + InternalTopicConfig topic = entry.getKey(); + List partitions = streamThread.restoreConsumer.partitionsFor(topic.name()); for (PartitionInfo partition : partitions) partitionInfos.put(new TopicPartition(partition.topic(), partition.partition()), partition); } @@ -241,8 +238,8 @@ private Map prepareTopic(Map assign(Cluster metadata, Map assign(Cluster metadata, Map> copartitionTopicGroups = streamThread.builder.copartitionGroups(); ensureCopartitioning(copartitionTopicGroups, internalSourceTopicGroups, - metadata.withPartitions(internalPartitionInfos)); + metadata.withPartitions(internalPartitionInfos)); internalPartitionInfos = prepareTopic(internalSourceTopicToTaskIds, false); @@ -460,10 +457,10 @@ public Map assign(Cluster metadata, Map value : values) { for (TopicPartition topicPartition : value) { topicToPartitionInfo.put(topicPartition, new PartitionInfo(topicPartition.topic(), - topicPartition.partition(), - null, - new Node[0], - new Node[0])); + topicPartition.partition(), + null, + new Node[0], + new Node[0])); } } metadataWithInternalTopics = Cluster.empty().withPartitions(topicToPartitionInfo); @@ -619,7 +616,7 @@ private void ensureCopartitioning(Set copartitionGroup, MapemptySet(), - Collections.emptySet()); + Collections.emptySet()); private final TaskId task0 = new TaskId(0, 0); private final TaskId task1 = new TaskId(0, 1); @@ -303,10 +303,13 @@ public void testAssignWithStates() throws Exception { String client1 = "client1"; - StreamThread thread10 = new StreamThread(builder, config, new MockClientSupplier(), applicationId, client1, uuid1, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); + MockClientSupplier mockClientSupplier = new MockClientSupplier(); + StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, applicationId, client1, uuid1, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); + partitionAssignor.configure(config.getConsumerConfigs(thread10, applicationId, client1)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); Map subscriptions = new HashMap<>(); subscriptions.put("consumer10", @@ -505,7 +508,7 @@ public void testAssignWithInternalTopics() throws Exception { StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, applicationId, client1)); - MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(clientSupplier.restoreConsumer); + MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(thread10.config, clientSupplier.restoreConsumer); partitionAssignor.setInternalTopicManager(internalTopicManager); Map subscriptions = new HashMap<>(); @@ -547,13 +550,13 @@ public void testAssignWithInternalTopicThatsSourceIsAnotherInternalTopic() throw StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, applicationId, client1)); - MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(clientSupplier.restoreConsumer); + MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(thread10.config, clientSupplier.restoreConsumer); partitionAssignor.setInternalTopicManager(internalTopicManager); Map subscriptions = new HashMap<>(); Set emptyTasks = Collections.emptySet(); subscriptions.put("consumer10", - new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, emptyTasks, emptyTasks, userEndPoint).encode())); + new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, emptyTasks, emptyTasks, userEndPoint).encode())); Map assignments = partitionAssignor.assign(metadata, subscriptions); @@ -642,7 +645,7 @@ public void shouldThrowExceptionIfApplicationServerConfigIsNotHostPortPair() thr final MockClientSupplier clientSupplier = new MockClientSupplier(); final StreamThread streamThread = new StreamThread(builder, config, clientSupplier, applicationId, client1, uuid1, - new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); + new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); @@ -653,7 +656,7 @@ public void shouldThrowExceptionIfApplicationServerConfigIsNotHostPortPair() thr // pass } } - + @Test public void shouldThrowExceptionIfApplicationServerConfigPortIsNotAnInteger() throws Exception { final Properties properties = configProps(); @@ -669,7 +672,7 @@ public void shouldThrowExceptionIfApplicationServerConfigPortIsNotAnInteger() th final MockClientSupplier clientSupplier = new MockClientSupplier(); final StreamThread streamThread = new StreamThread(builder, config, clientSupplier, applicationId, client1, uuid1, - new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); + new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); @@ -702,10 +705,10 @@ public void shouldSetClusterMetadataOnAssignment() throws Exception { final List topic = Arrays.asList(new TopicPartition("topic", 0)); final Map> hostState = Collections.singletonMap(new HostInfo("localhost", 80), - Collections.singleton(new TopicPartition("topic", 0))); + Collections.singleton(new TopicPartition("topic", 0))); final AssignmentInfo assignmentInfo = new AssignmentInfo(Collections.singletonList(new TaskId(0, 0)), - Collections.>emptyMap(), - hostState); + Collections.>emptyMap(), + hostState); partitionAssignor.onAssignment(new PartitionAssignor.Assignment(topic, assignmentInfo.encode())); @@ -731,8 +734,8 @@ private class MockInternalTopicManager extends InternalTopicManager { public Map readyTopics = new HashMap<>(); public MockConsumer restoreConsumer; - public MockInternalTopicManager(MockConsumer restoreConsumer) { - super(); + public MockInternalTopicManager(StreamsConfig streamsConfig, MockConsumer restoreConsumer) { + super(new StreamsKafkaClient(streamsConfig), 0, 0); this.restoreConsumer = restoreConsumer; } From 8e064f1863aa8095cfae3f11a6f85513c498e479 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Fri, 7 Oct 2016 14:36:12 -0700 Subject: [PATCH 10/16] added the missing class. --- .../internals/StreamsKafkaClient.java | 366 ++++++++++++++++++ 1 file changed, 366 insertions(+) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java new file mode 100644 index 000000000000..01784b54f68d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -0,0 +1,366 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package org.apache.kafka.streams.processor.internals; + +import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.NetworkClient; +import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.metrics.JmxReporter; +import org.apache.kafka.common.metrics.MetricConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.MetricsReporter; +import org.apache.kafka.common.network.ChannelBuilder; +import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.CreateTopicsRequest; +import org.apache.kafka.common.requests.CreateTopicsResponse; +import org.apache.kafka.common.requests.DeleteTopicsRequest; +import org.apache.kafka.common.requests.DeleteTopicsResponse; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsException; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.HashSet; +import java.util.List; +import java.util.LinkedHashMap; +import java.util.Properties; +import java.util.Collection; +import java.util.concurrent.TimeUnit; + +public class StreamsKafkaClient { + + private final KafkaClient kafkaClient; + private final List reporters; + private final StreamsConfig streamsConfig; + + private static final int MAX_INFLIGHT_REQUESTS = 100; + private static final long MAX_WAIT_TIME_MS = 30000; + + public StreamsKafkaClient(final StreamsConfig streamsConfig) { + + this.streamsConfig = streamsConfig; + final Time time = new SystemTime(); + + final Map metricTags = new LinkedHashMap<>(); + metricTags.put("client-id", StreamsConfig.CLIENT_ID_CONFIG); + + final Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); + final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); + metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); + + final MetricConfig metricConfig = new MetricConfig().samples(streamsConfig.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG)) + .timeWindow(streamsConfig.getLong(CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS) + .tags(metricTags); + reporters = streamsConfig.getConfiguredInstances(ProducerConfig.METRIC_REPORTER_CLASSES_CONFIG, + MetricsReporter.class); + // TODO: This should come from the KafkaStream + reporters.add(new JmxReporter("kafka.admin")); + final Metrics metrics = new Metrics(metricConfig, reporters, time); + + final ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(streamsConfig.values()); + + final Selector selector = new Selector(streamsConfig.getLong(StreamsConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, "kafka-client", channelBuilder); + + kafkaClient = new NetworkClient( + selector, + metadata, + streamsConfig.getString(StreamsConfig.CLIENT_ID_CONFIG), + MAX_INFLIGHT_REQUESTS, // a fixed large enough value will suffice + streamsConfig.getLong(StreamsConfig.RECONNECT_BACKOFF_MS_CONFIG), + streamsConfig.getInt(StreamsConfig.SEND_BUFFER_CONFIG), + streamsConfig.getInt(StreamsConfig.RECEIVE_BUFFER_CONFIG), + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG), time); + } + + public void close() throws IOException { + for (MetricsReporter metricsReporter: this.reporters) { + metricsReporter.close(); + } + } + + /** + * Cretes a new topic with the given number of partitions and replication factor. + * + * @param internalTopicConfig + * @param numPartitions + * @param replicationFactor + */ + public void createTopic(final InternalTopicConfig internalTopicConfig, final int numPartitions, final int replicationFactor, final long windowChangeLogAdditionalRetention) { + Map topicsMap = new HashMap<>(); + topicsMap.put(internalTopicConfig, numPartitions); + createTopics(topicsMap, replicationFactor, windowChangeLogAdditionalRetention); + } + + /** + * Creates a set of new topics using batch request. + * + * @param topicsMap + * @param replicationFactor + * @param windowChangeLogAdditionalRetention + */ + public void createTopics(final Map topicsMap, final int replicationFactor, final long windowChangeLogAdditionalRetention) { + + final Map topicRequestDetails = new HashMap<>(); + for (InternalTopicConfig internalTopicConfig:topicsMap.keySet()) { + final Properties topicProperties = internalTopicConfig.toProperties(windowChangeLogAdditionalRetention); + final Map topicConfig = new HashMap<>(); + for (String key : topicProperties.stringPropertyNames()) { + topicConfig.put(key, topicProperties.getProperty(key)); + } + final CreateTopicsRequest.TopicDetails topicDetails = new CreateTopicsRequest.TopicDetails(topicsMap.get(internalTopicConfig), (short) replicationFactor, topicConfig); + + topicRequestDetails.put(internalTopicConfig.name(), topicDetails); + } + final CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest(topicRequestDetails, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); + final ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS); + final CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); + for (InternalTopicConfig internalTopicConfig:topicsMap.keySet()) { + if (createTopicsResponse.errors().get(internalTopicConfig.name()).code() > 0) { + throw new StreamsException("Could not create topic: " + internalTopicConfig.name() + ". " + createTopicsResponse.errors().get(internalTopicConfig.name()).name()); + } + } + } + + + /** + * Delete a given topic. + * + * @param internalTopicConfig + */ + public void deleteTopic(final InternalTopicConfig internalTopicConfig) { + final Set topics = new HashSet(); + topics.add(internalTopicConfig.name()); + final DeleteTopicsRequest deleteTopicsRequest = new DeleteTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); + final ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS); + final DeleteTopicsResponse deleteTopicsResponse = new DeleteTopicsResponse(clientResponse.responseBody()); + if (deleteTopicsResponse.errors().get(internalTopicConfig.name()).code() > 0) { + throw new StreamsException("Could not delete topic: " + internalTopicConfig.name()); + } + } + + /** + * Delets a set of topics. + * + * @param topics + */ + public void deleteTopics(final Map topics) { + + final Set topicNames = new HashSet<>(); + for (InternalTopicConfig internalTopicConfig: topics.keySet()) { + topicNames.add(internalTopicConfig.name()); + } + deleteTopics(topicNames); + } + + /** + * Delete a set of topics in one request. + * + * @param topics + */ + public void deleteTopics(final Set topics) { + + final DeleteTopicsRequest deleteTopicsRequest = new DeleteTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); + final ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS); + final DeleteTopicsResponse deleteTopicsResponse = new DeleteTopicsResponse(clientResponse.responseBody()); + for (String topicName: deleteTopicsResponse.errors().keySet()) { + if (deleteTopicsResponse.errors().get(topicName).code() > 0) { + throw new StreamsException("Could not delete topic: " + topicName); + } + } + + } + + /** + * Send a request to kafka broker of this client. Keep polling until the corresponding response is received. + * + * @param request + * @param apiKeys + */ + private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) { + + String brokerId = null; + final SystemTime systemTime = new SystemTime(); + + final Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); + final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); + metadata.update(Cluster.bootstrap(addresses), systemTime.milliseconds()); + + final List nodes = metadata.fetch().nodes(); + final long readyTimeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; + boolean foundNode = false; + while (!foundNode && (systemTime.milliseconds() < readyTimeout)) { + for (Node node: nodes) { + if (kafkaClient.ready(node, systemTime.milliseconds())) { + brokerId = Integer.toString(node.id()); + foundNode = true; + break; + } + } + kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); + } + if (brokerId == null) { + throw new StreamsException("Could not find any available broker."); + } + + final RequestSend send = new RequestSend(brokerId, + kafkaClient.nextRequestHeader(apiKeys), + request); + + final ClientRequest clientRequest = new ClientRequest(systemTime.milliseconds(), true, send, null); + + kafkaClient.send(clientRequest, systemTime.milliseconds()); + + final long responseTimeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; + // Poll for the response. + while (systemTime.milliseconds() < responseTimeout) { + List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); + if (!responseList.isEmpty()) { + if (responseList.size() > 1) { + throw new StreamsException("Sent one request but received multiple or no responses."); + } + if (responseList.get(0).request().equals(clientRequest)) { + return responseList.get(0); + } else { + throw new StreamsException("Inconsistent response received."); + } + } + } + throw new StreamsException("Failed to get response from broker within timeout"); + } + + + /** + * Get the metadata for a topic. + * @param topic + * @return + */ + public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { + + final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); + final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); + for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { + if (topicMetadata.topic().equalsIgnoreCase(topic)) { + return topicMetadata; + } + } + return null; + } + + /** + * Check to see if a topic exists. + * @param topicName + * @return + */ + public boolean topicExists(final String topicName) { + + if (getTopicMetadata(topicName) != null) { + return true; + } + return false; + } + + public Set filterValidateExistingTopics(final Map topicsPartitionsMap) { + Set nonExistingTopics = new HashSet<>(); + final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); + final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); + // The names of existing topics + Map existingTopicNamesPartitions = new HashMap<>(); + for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { + existingTopicNamesPartitions.put(topicMetadata.topic(), topicMetadata.partitionMetadata().size()); + } + // Add the topics that don't exist to the nonExistingTopics. + for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { + if (existingTopicNamesPartitions.get(topic.name()) == null) { + nonExistingTopics.add(topic); + } else if (existingTopicNamesPartitions.get(topic.name()) != topicsPartitionsMap.get(topic.name())) { + + } + } + + return nonExistingTopics; + } + + /** + * Return the non existing topics. + * + * @param topicsPartitionsMap + * @param topicsMetadata + * @return + */ + public Map filterExistingTopics(final Map topicsPartitionsMap, Collection topicsMetadata) { + // The names of existing topics + Map existingTopicNamesPartitions = new HashMap<>(); + for (MetadataResponse.TopicMetadata topicMetadata: topicsMetadata) { + existingTopicNamesPartitions.put(topicMetadata.topic(), topicMetadata.partitionMetadata().size()); + } + Map nonExistingTopics = new HashMap<>(); + // Add the topics that don't exist to the nonExistingTopics. + for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { + if (existingTopicNamesPartitions.get(topic.name()) == null) { + nonExistingTopics.put(topic, topicsPartitionsMap.get(topic)); + } + } + return nonExistingTopics; + } + + /** + * Return the topics that exist but have different partiton number to be deleted. + * @param topicsPartitionsMap + * @param topicsMetadata + * @return + */ + public Map getTopicsToBeDeleted(final Map topicsPartitionsMap, Collection topicsMetadata) { + // The names of existing topics + Map existingTopicNamesPartitions = new HashMap<>(); + for (MetadataResponse.TopicMetadata topicMetadata: topicsMetadata) { + existingTopicNamesPartitions.put(topicMetadata.topic(), topicMetadata.partitionMetadata().size()); + } + Map deleteTopics = new HashMap<>(); + // Add the topics that don't exist to the nonExistingTopics. + for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { + if (existingTopicNamesPartitions.get(topic.name()) != null) { + if (existingTopicNamesPartitions.get(topic.name()) != topicsPartitionsMap.get(topic)) { + deleteTopics.put(topic, topicsPartitionsMap.get(topic)); + } + } + } + return deleteTopics; + } + + + public Collection fetchTopicMetadata() { + final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); + final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); + return metadataResponse.topicMetadata(); + } + +} From 37079dd28a15a6d36632631227c9d718ca451745 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Sun, 9 Oct 2016 20:09:33 -0700 Subject: [PATCH 11/16] Fixed the issue with tests. --- .../StreamPartitionAssignorTest.java | 53 ++++++--------- .../processor/internals/StreamThreadTest.java | 20 ++++-- .../kafka/test/MockInternalTopicManager.java | 68 +++++++++++++++++++ 3 files changed, 100 insertions(+), 41 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index 05e2cd6b21eb..bb18e3661d9a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.internals.PartitionAssignor; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; @@ -35,21 +34,22 @@ import org.apache.kafka.streams.state.HostInfo; import org.apache.kafka.test.MockClientSupplier; import org.apache.kafka.test.MockProcessorSupplier; -import org.apache.kafka.test.MockStateStoreSupplier; import org.apache.kafka.test.MockTimestampExtractor; +import org.apache.kafka.test.MockStateStoreSupplier; +import org.apache.kafka.test.MockInternalTopicManager; import org.junit.Assert; import org.junit.Test; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; -import java.util.Map; -import java.util.Properties; import java.util.Set; +import java.util.Properties; +import java.util.Map; import java.util.UUID; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Collections; +import java.util.ArrayList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -169,10 +169,12 @@ public void testAssignBasic() throws Exception { UUID uuid2 = UUID.randomUUID(); String client1 = "client1"; - StreamThread thread10 = new StreamThread(builder, config, new MockClientSupplier(), "test", client1, uuid1, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); + MockClientSupplier mockClientSupplier = new MockClientSupplier(); + StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, "test", client1, uuid1, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); Map subscriptions = new HashMap<>(); subscriptions.put("consumer10", @@ -235,10 +237,12 @@ public void testAssignWithNewTasks() throws Exception { UUID uuid2 = UUID.randomUUID(); String client1 = "client1"; - StreamThread thread10 = new StreamThread(builder, config, new MockClientSupplier(), "test", client1, uuid1, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); + MockClientSupplier mockClientSupplier = new MockClientSupplier(); + StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, "test", client1, uuid1, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); Map subscriptions = new HashMap<>(); subscriptions.put("consumer10", @@ -361,10 +365,13 @@ public void testAssignWithStandbyReplicas() throws Exception { UUID uuid2 = UUID.randomUUID(); String client1 = "client1"; + MockClientSupplier mockClientSupplier = new MockClientSupplier(); + StreamThread thread10 = new StreamThread(builder, config, new MockClientSupplier(), "test", client1, uuid1, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); Map subscriptions = new HashMap<>(); subscriptions.put("consumer10", @@ -615,6 +622,7 @@ public void shouldMapUserEndPointToTopicPartitions() throws Exception { final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client1)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamThread.config, clientSupplier.restoreConsumer)); final Map subscriptions = new HashMap<>(); final Set emptyTasks = Collections.emptySet(); @@ -648,6 +656,7 @@ public void shouldThrowExceptionIfApplicationServerConfigIsNotHostPortPair() thr new Metrics(), new SystemTime(), new StreamsMetadataState(builder)); final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamThread.config, clientSupplier.restoreConsumer)); try { partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client1)); @@ -728,28 +737,4 @@ public void shouldReturnEmptyClusterMetadataIfItHasntBeenBuilt() throws Exceptio } - - private class MockInternalTopicManager extends InternalTopicManager { - - public Map readyTopics = new HashMap<>(); - public MockConsumer restoreConsumer; - - public MockInternalTopicManager(StreamsConfig streamsConfig, MockConsumer restoreConsumer) { - super(new StreamsKafkaClient(streamsConfig), 0, 0); - - this.restoreConsumer = restoreConsumer; - } - - @Override - public void makeReady(InternalTopicConfig topic, int numPartitions) { - readyTopics.put(topic.name(), numPartitions); - - List partitions = new ArrayList<>(); - for (int i = 0; i < numPartitions; i++) { - partitions.add(new PartitionInfo(topic.name(), i, null, null, null)); - } - - restoreConsumer.updatePartitions(topic.name(), partitions); - } - } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 2f252e9fa605..24cc03b20e5f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -38,6 +38,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TopologyBuilder; import org.apache.kafka.test.MockClientSupplier; +import org.apache.kafka.test.MockInternalTopicManager; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Test; @@ -165,7 +166,8 @@ public void testPartitionAssignmentChange() throws Exception { builder.addSource("source3", "topic3"); builder.addProcessor("processor", new MockProcessorSupplier(), "source2", "source3"); - StreamThread thread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, clientId, processId, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)) { + MockClientSupplier mockClientSupplier = new MockClientSupplier(); + StreamThread thread = new StreamThread(builder, config, mockClientSupplier, applicationId, clientId, processId, new Metrics(), new SystemTime(), new StreamsMetadataState(builder)) { @Override protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) { ProcessorTopology topology = builder.build(id.topicGroupId); @@ -173,7 +175,7 @@ protected StreamTask createStreamTask(TaskId id, Collection part } }; - initPartitionGrouper(config, thread); + initPartitionGrouper(config, thread, mockClientSupplier); ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; @@ -284,7 +286,8 @@ public void testMaybeClean() throws Exception { TopologyBuilder builder = new TopologyBuilder().setApplicationId("X"); builder.addSource("source1", "topic1"); - StreamThread thread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, clientId, processId, new Metrics(), mockTime, new StreamsMetadataState(builder)) { + MockClientSupplier mockClientSupplier = new MockClientSupplier(); + StreamThread thread = new StreamThread(builder, config, mockClientSupplier, applicationId, clientId, processId, new Metrics(), mockTime, new StreamsMetadataState(builder)) { @Override public void maybeClean() { super.maybeClean(); @@ -297,7 +300,7 @@ protected StreamTask createStreamTask(TaskId id, Collection part } }; - initPartitionGrouper(config, thread); + initPartitionGrouper(config, thread, mockClientSupplier); ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; @@ -403,7 +406,8 @@ public void testMaybeCommit() throws Exception { TopologyBuilder builder = new TopologyBuilder().setApplicationId("X"); builder.addSource("source1", "topic1"); - StreamThread thread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, clientId, processId, new Metrics(), mockTime, new StreamsMetadataState(builder)) { + MockClientSupplier mockClientSupplier = new MockClientSupplier(); + StreamThread thread = new StreamThread(builder, config, mockClientSupplier, applicationId, clientId, processId, new Metrics(), mockTime, new StreamsMetadataState(builder)) { @Override public void maybeCommit() { super.maybeCommit(); @@ -416,7 +420,7 @@ protected StreamTask createStreamTask(TaskId id, Collection part } }; - initPartitionGrouper(config, thread); + initPartitionGrouper(config, thread, mockClientSupplier); ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; @@ -481,11 +485,13 @@ public void testInjectClients() { assertSame(clientSupplier.restoreConsumer, thread.restoreConsumer); } - private void initPartitionGrouper(StreamsConfig config, StreamThread thread) { + private void initPartitionGrouper(StreamsConfig config, StreamThread thread, MockClientSupplier clientSupplier) { StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread, thread.applicationId, thread.clientId)); + MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(thread.config, clientSupplier.restoreConsumer); + partitionAssignor.setInternalTopicManager(internalTopicManager); Map assignments = partitionAssignor.assign(metadata, Collections.singletonMap("client", subscription)); diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java new file mode 100644 index 000000000000..db40059b1586 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java @@ -0,0 +1,68 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.test; + +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.processor.internals.InternalTopicConfig; +import org.apache.kafka.streams.processor.internals.InternalTopicManager; +import org.apache.kafka.streams.processor.internals.StreamsKafkaClient; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MockInternalTopicManager extends InternalTopicManager { + + public Map readyTopics = new HashMap<>(); + public MockConsumer restoreConsumer; + + public MockInternalTopicManager(StreamsConfig streamsConfig, MockConsumer restoreConsumer) { + super(new StreamsKafkaClient(streamsConfig), 0, 0); + + this.restoreConsumer = restoreConsumer; + } + + @Override + public void makeReady(InternalTopicConfig topic, int numPartitions) { + readyTopics.put(topic.name(), numPartitions); + + List partitions = new ArrayList<>(); + for (int i = 0; i < numPartitions; i++) { + partitions.add(new PartitionInfo(topic.name(), i, null, null, null)); + } + + restoreConsumer.updatePartitions(topic.name(), partitions); + } + + @Override + public void makeReady(final Map topics) { + for (InternalTopicConfig topic: topics.keySet()) { + int numPartitions = topics.get(topic); + readyTopics.put(topic.name(), numPartitions); + List partitions = new ArrayList<>(); + for (int i = 0; i < numPartitions; i++) { + partitions.add(new PartitionInfo(topic.name(), i, null, null, null)); + } + restoreConsumer.updatePartitions(topic.name(), partitions); + } + + } +} \ No newline at end of file From 21b6907d457591ba0a242e548549d315b38e69f2 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Mon, 24 Oct 2016 09:46:49 -0700 Subject: [PATCH 12/16] Made minor changes based on Guozhang's feedback. --- gradlew | 5 - gradlew.bat | 6 + .../internals/InternalTopicManager.java | 52 +++++++- .../internals/StreamsKafkaClient.java | 118 +----------------- .../InternalTopicIntegrationTest.java | 2 +- 5 files changed, 60 insertions(+), 123 deletions(-) diff --git a/gradlew b/gradlew index 9aa616c273d8..27309d92314c 100755 --- a/gradlew +++ b/gradlew @@ -161,9 +161,4 @@ function splitJvmOpts() { eval splitJvmOpts $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS JVM_OPTS[${#JVM_OPTS[*]}]="-Dorg.gradle.appname=$APP_BASE_NAME" -# by default we should be in the correct project dir, but when run from Finder on Mac, the cwd is wrong -if [[ "$(uname)" == "Darwin" ]] && [[ "$HOME" == "$PWD" ]]; then - cd "$(dirname "$0")" -fi - exec "$JAVACMD" "${JVM_OPTS[@]}" -classpath "$CLASSPATH" org.gradle.wrapper.GradleWrapperMain "$@" diff --git a/gradlew.bat b/gradlew.bat index e95643d6a2ca..f6d5974e72fd 100644 --- a/gradlew.bat +++ b/gradlew.bat @@ -49,6 +49,7 @@ goto fail @rem Get command-line arguments, handling Windows variants if not "%OS%" == "Windows_NT" goto win9xME_args +if "%@eval[2+2]" == "4" goto 4NT_args :win9xME_args @rem Slurp the command line arguments. @@ -59,6 +60,11 @@ set _SKIP=2 if "x%~1" == "x" goto execute set CMD_LINE_ARGS=%* +goto execute + +:4NT_args +@rem Get arguments from the 4NT Shell from JP Software +set CMD_LINE_ARGS=%$ :execute @rem Setup the command line diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 63d24d0d0f09..ac9d5b2bcfed 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -57,8 +57,8 @@ public void makeReady(final Map topics) { //TODO: Add loop/ inspect th error codes Collection topicMetadatas = streamsKafkaClient.fetchTopicMetadata(); - Map topicsToBeDeleted = streamsKafkaClient.getTopicsToBeDeleted(topics, topicMetadatas); - Map topicsToBeCreated = streamsKafkaClient.filterExistingTopics(topics, topicMetadatas); + Map topicsToBeDeleted = getTopicsToBeDeleted(topics, topicMetadatas); + Map topicsToBeCreated = filterExistingTopics(topics, topicMetadatas); topicsToBeCreated.putAll(topicsToBeDeleted); streamsKafkaClient.deleteTopics(topicsToBeDeleted); streamsKafkaClient.createTopics(topicsToBeCreated, replicationFactor, windowChangeLogAdditionalRetention); @@ -68,4 +68,52 @@ public void close() throws IOException { streamsKafkaClient.close(); } + /** + * Return the non existing topics. + * + * @param topicsPartitionsMap + * @param topicsMetadata + * @return + */ + public Map filterExistingTopics(final Map topicsPartitionsMap, Collection topicsMetadata) { + Map existingTopicNamesPartitions = getExistingTopicNamesPartitions(topicsMetadata); + Map nonExistingTopics = new HashMap<>(); + // Add the topics that don't exist to the nonExistingTopics. + for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { + if (existingTopicNamesPartitions.get(topic.name()) == null) { + nonExistingTopics.put(topic, topicsPartitionsMap.get(topic)); + } + } + return nonExistingTopics; + } + + /** + * Return the topics that exist but have different partiton number to be deleted. + * @param topicsPartitionsMap + * @param topicsMetadata + * @return + */ + public Map getTopicsToBeDeleted(final Map topicsPartitionsMap, Collection topicsMetadata) { + Map existingTopicNamesPartitions = getExistingTopicNamesPartitions(topicsMetadata); + Map deleteTopics = new HashMap<>(); + // Add the topics that don't exist to the nonExistingTopics. + for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { + if (existingTopicNamesPartitions.get(topic.name()) != null) { + if (existingTopicNamesPartitions.get(topic.name()) != topicsPartitionsMap.get(topic)) { + deleteTopics.put(topic, topicsPartitionsMap.get(topic)); + } + } + } + return deleteTopics; + } + + private Map getExistingTopicNamesPartitions(Collection topicsMetadata) { + // The names of existing topics + Map existingTopicNamesPartitions = new HashMap<>(); + for (MetadataResponse.TopicMetadata topicMetadata: topicsMetadata) { + existingTopicNamesPartitions.put(topicMetadata.topic(), topicMetadata.partitionMetadata().size()); + } + return existingTopicNamesPartitions; + } + } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 01784b54f68d..2998116dcbc1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -63,7 +63,6 @@ public class StreamsKafkaClient { private final StreamsConfig streamsConfig; private static final int MAX_INFLIGHT_REQUESTS = 100; - private static final long MAX_WAIT_TIME_MS = 30000; public StreamsKafkaClient(final StreamsConfig streamsConfig) { @@ -107,19 +106,6 @@ public void close() throws IOException { } } - /** - * Cretes a new topic with the given number of partitions and replication factor. - * - * @param internalTopicConfig - * @param numPartitions - * @param replicationFactor - */ - public void createTopic(final InternalTopicConfig internalTopicConfig, final int numPartitions, final int replicationFactor, final long windowChangeLogAdditionalRetention) { - Map topicsMap = new HashMap<>(); - topicsMap.put(internalTopicConfig, numPartitions); - createTopics(topicsMap, replicationFactor, windowChangeLogAdditionalRetention); - } - /** * Creates a set of new topics using batch request. * @@ -150,23 +136,6 @@ public void createTopics(final Map topicsMap, fina } } - - /** - * Delete a given topic. - * - * @param internalTopicConfig - */ - public void deleteTopic(final InternalTopicConfig internalTopicConfig) { - final Set topics = new HashSet(); - topics.add(internalTopicConfig.name()); - final DeleteTopicsRequest deleteTopicsRequest = new DeleteTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - final ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS); - final DeleteTopicsResponse deleteTopicsResponse = new DeleteTopicsResponse(clientResponse.responseBody()); - if (deleteTopicsResponse.errors().get(internalTopicConfig.name()).code() > 0) { - throw new StreamsException("Could not delete topic: " + internalTopicConfig.name()); - } - } - /** * Delets a set of topics. * @@ -186,7 +155,7 @@ public void deleteTopics(final Map topics) { * * @param topics */ - public void deleteTopics(final Set topics) { + private void deleteTopics(final Set topics) { final DeleteTopicsRequest deleteTopicsRequest = new DeleteTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); final ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS); @@ -215,7 +184,7 @@ private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) metadata.update(Cluster.bootstrap(addresses), systemTime.milliseconds()); final List nodes = metadata.fetch().nodes(); - final long readyTimeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; + final long readyTimeout = systemTime.milliseconds() + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG); boolean foundNode = false; while (!foundNode && (systemTime.milliseconds() < readyTimeout)) { for (Node node: nodes) { @@ -239,7 +208,7 @@ private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) kafkaClient.send(clientRequest, systemTime.milliseconds()); - final long responseTimeout = systemTime.milliseconds() + MAX_WAIT_TIME_MS; + final long responseTimeout = systemTime.milliseconds() + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG); // Poll for the response. while (systemTime.milliseconds() < responseTimeout) { List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); @@ -275,87 +244,6 @@ public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { return null; } - /** - * Check to see if a topic exists. - * @param topicName - * @return - */ - public boolean topicExists(final String topicName) { - - if (getTopicMetadata(topicName) != null) { - return true; - } - return false; - } - - public Set filterValidateExistingTopics(final Map topicsPartitionsMap) { - Set nonExistingTopics = new HashSet<>(); - final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); - final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); - // The names of existing topics - Map existingTopicNamesPartitions = new HashMap<>(); - for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { - existingTopicNamesPartitions.put(topicMetadata.topic(), topicMetadata.partitionMetadata().size()); - } - // Add the topics that don't exist to the nonExistingTopics. - for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { - if (existingTopicNamesPartitions.get(topic.name()) == null) { - nonExistingTopics.add(topic); - } else if (existingTopicNamesPartitions.get(topic.name()) != topicsPartitionsMap.get(topic.name())) { - - } - } - - return nonExistingTopics; - } - - /** - * Return the non existing topics. - * - * @param topicsPartitionsMap - * @param topicsMetadata - * @return - */ - public Map filterExistingTopics(final Map topicsPartitionsMap, Collection topicsMetadata) { - // The names of existing topics - Map existingTopicNamesPartitions = new HashMap<>(); - for (MetadataResponse.TopicMetadata topicMetadata: topicsMetadata) { - existingTopicNamesPartitions.put(topicMetadata.topic(), topicMetadata.partitionMetadata().size()); - } - Map nonExistingTopics = new HashMap<>(); - // Add the topics that don't exist to the nonExistingTopics. - for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { - if (existingTopicNamesPartitions.get(topic.name()) == null) { - nonExistingTopics.put(topic, topicsPartitionsMap.get(topic)); - } - } - return nonExistingTopics; - } - - /** - * Return the topics that exist but have different partiton number to be deleted. - * @param topicsPartitionsMap - * @param topicsMetadata - * @return - */ - public Map getTopicsToBeDeleted(final Map topicsPartitionsMap, Collection topicsMetadata) { - // The names of existing topics - Map existingTopicNamesPartitions = new HashMap<>(); - for (MetadataResponse.TopicMetadata topicMetadata: topicsMetadata) { - existingTopicNamesPartitions.put(topicMetadata.topic(), topicMetadata.partitionMetadata().size()); - } - Map deleteTopics = new HashMap<>(); - // Add the topics that don't exist to the nonExistingTopics. - for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { - if (existingTopicNamesPartitions.get(topic.name()) != null) { - if (existingTopicNamesPartitions.get(topic.name()) != topicsPartitionsMap.get(topic)) { - deleteTopics.put(topic, topicsPartitionsMap.get(topic)); - } - } - } - return deleteTopics; - } - public Collection fetchTopicMetadata() { final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java index 13c71820035b..0cbadfdc054e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java @@ -91,7 +91,7 @@ public void before() { private Properties getTopicConfigProperties(final String changelog) { // Note: You must initialize the ZkClient with ZKStringSerializer. If you don't, then - // createTopic() will only seem to work (it will return without error). The topic will exist in + // createTopics() will only seem to work (it will return without error). The topic will exist in // only ZooKeeper and will be returned when listing topics, but Kafka itself does not create the // topic. final ZkClient zkClient = new ZkClient( From 253784edc252d80d5ecb3eda8ef506657f659ba1 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Mon, 24 Oct 2016 13:35:48 -0700 Subject: [PATCH 13/16] Minor change to ignore unnecessary error handling. --- .../processor/internals/StreamsKafkaClient.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 2998116dcbc1..8933829679d4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -31,6 +31,7 @@ import org.apache.kafka.common.network.ChannelBuilder; import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.requests.CreateTopicsResponse; @@ -130,8 +131,13 @@ public void createTopics(final Map topicsMap, fina final ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS); final CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); for (InternalTopicConfig internalTopicConfig:topicsMap.keySet()) { - if (createTopicsResponse.errors().get(internalTopicConfig.name()).code() > 0) { - throw new StreamsException("Could not create topic: " + internalTopicConfig.name() + ". " + createTopicsResponse.errors().get(internalTopicConfig.name()).name()); + short errorCode = createTopicsResponse.errors().get(internalTopicConfig.name()).code(); + if (errorCode > 0) { + if (errorCode == Errors.TOPIC_ALREADY_EXISTS.code()) { + continue; + } else { + throw new StreamsException("Could not create topic: " + internalTopicConfig.name() + ". " + createTopicsResponse.errors().get(internalTopicConfig.name()).name()); + } } } } From 3599534a55347aad55957ced02e0901082416e5f Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Tue, 25 Oct 2016 15:32:44 -0700 Subject: [PATCH 14/16] Added a retry look for internal topic creation. --- .../processor/internals/InternalTopicManager.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index ac9d5b2bcfed..f67fa4434b42 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.streams.errors.StreamsException; import java.io.IOException; import java.util.Collection; @@ -30,6 +31,7 @@ public class InternalTopicManager { public static final String CLEANUP_POLICY_PROP = "cleanup.policy"; public static final String RETENTION_MS = "retention.ms"; public static final Long WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS); + private static final int MAX_TOPIC_READY_TRY = 5; private final long windowChangeLogAdditionalRetention; @@ -55,13 +57,19 @@ public void makeReady(final InternalTopicConfig topic, int numPartitions) { */ public void makeReady(final Map topics) { - //TODO: Add loop/ inspect th error codes Collection topicMetadatas = streamsKafkaClient.fetchTopicMetadata(); Map topicsToBeDeleted = getTopicsToBeDeleted(topics, topicMetadatas); Map topicsToBeCreated = filterExistingTopics(topics, topicMetadatas); topicsToBeCreated.putAll(topicsToBeDeleted); streamsKafkaClient.deleteTopics(topicsToBeDeleted); - streamsKafkaClient.createTopics(topicsToBeCreated, replicationFactor, windowChangeLogAdditionalRetention); + for (int i = 0; i < MAX_TOPIC_READY_TRY; i++) { + try { + streamsKafkaClient.createTopics(topicsToBeCreated, replicationFactor, windowChangeLogAdditionalRetention); + return; + } catch (StreamsException ex) { + } + } + throw new StreamsException("Could not create internal topics."); } public void close() throws IOException { From 6ef4cd1a01af6af21081acb03775566305a974a5 Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Mon, 28 Nov 2016 16:28:39 -0800 Subject: [PATCH 15/16] Refactored the StreamsKafkaClient based on the changes in the client protocol. --- .../internals/StreamsKafkaClient.java | 43 ++++++++++++------- 1 file changed, 27 insertions(+), 16 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index c11da642a2f6..2b78d1f4f800 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -32,14 +32,14 @@ import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.requests.CreateTopicsResponse; import org.apache.kafka.common.requests.DeleteTopicsRequest; import org.apache.kafka.common.requests.DeleteTopicsResponse; -import org.apache.kafka.common.requests.Req; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; @@ -128,8 +128,12 @@ public void createTopics(final Map topicsMap, fina topicRequestDetails.put(internalTopicConfig.name(), topicDetails); } final CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest(topicRequestDetails, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - final ClientResponse clientResponse = sendRequest(createTopicsRequest.toStruct(), ApiKeys.CREATE_TOPICS); - final CreateTopicsResponse createTopicsResponse = new CreateTopicsResponse(clientResponse.responseBody()); + final ClientResponse clientResponse = sendRequest(createTopicsRequest, ApiKeys.CREATE_TOPICS); + if (!(clientResponse.responseBody() instanceof CreateTopicsResponse)) { + throw new StreamsException("Inconsistent response type for internal topic creation request. Expected CreateTopicsResponse but received " + clientResponse.responseBody().getClass().getName()); + } + final CreateTopicsResponse createTopicsResponse = (CreateTopicsResponse) clientResponse.responseBody(); + for (InternalTopicConfig internalTopicConfig:topicsMap.keySet()) { short errorCode = createTopicsResponse.errors().get(internalTopicConfig.name()).code(); if (errorCode > 0) { @@ -164,8 +168,11 @@ public void deleteTopics(final Map topics) { private void deleteTopics(final Set topics) { final DeleteTopicsRequest deleteTopicsRequest = new DeleteTopicsRequest(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - final ClientResponse clientResponse = sendRequest(deleteTopicsRequest.toStruct(), ApiKeys.DELETE_TOPICS); - final DeleteTopicsResponse deleteTopicsResponse = new DeleteTopicsResponse(clientResponse.responseBody()); + final ClientResponse clientResponse = sendRequest(deleteTopicsRequest, ApiKeys.DELETE_TOPICS); + if (!(clientResponse.responseBody() instanceof DeleteTopicsResponse)) { + throw new StreamsException("Inconsistent response type for internal topic deletion request. Expected DeleteTopicsResponse but received " + clientResponse.responseBody().getClass().getName()); + } + final DeleteTopicsResponse deleteTopicsResponse = (DeleteTopicsResponse) clientResponse.responseBody(); for (String topicName: deleteTopicsResponse.errors().keySet()) { if (deleteTopicsResponse.errors().get(topicName).code() > 0) { throw new StreamsException("Could not delete topic: " + topicName); @@ -180,7 +187,7 @@ private void deleteTopics(final Set topics) { * @param request * @param apiKeys */ - private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) { + private ClientResponse sendRequest(final AbstractRequest request, final ApiKeys apiKeys) { String brokerId = null; final SystemTime systemTime = new SystemTime(); @@ -206,11 +213,9 @@ private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) throw new StreamsException("Could not find any available broker."); } - final RequestSend send = new RequestSend(brokerId, - kafkaClient.nextRequestHeader(apiKeys), - request); + final RequestHeader requestHeader = kafkaClient.nextRequestHeader(apiKeys); - final ClientRequest clientRequest = new ClientRequest(systemTime.milliseconds(), true, send, null); + final ClientRequest clientRequest = new ClientRequest(brokerId, systemTime.milliseconds(), true, requestHeader, request, null); kafkaClient.send(clientRequest, systemTime.milliseconds()); @@ -222,7 +227,7 @@ private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) if (responseList.size() > 1) { throw new StreamsException("Sent one request but received multiple or no responses."); } - if (responseList.get(0).request().equals(clientRequest)) { + if (responseList.get(0).requestHeader().equals(requestHeader)) { return responseList.get(0); } else { throw new StreamsException("Inconsistent response received."); @@ -240,8 +245,11 @@ private ClientResponse sendRequest(final Struct request, final ApiKeys apiKeys) */ public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { - final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); - final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); + final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics(), ApiKeys.METADATA); + if (!(clientResponse.responseBody() instanceof MetadataResponse)) { + throw new StreamsException("Inconsistent response type for internal topic metadata request. Expected MetadataResponse but received " + clientResponse.responseBody().getClass().getName()); + } + final MetadataResponse metadataResponse = (MetadataResponse) clientResponse.responseBody(); for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { if (topicMetadata.topic().equalsIgnoreCase(topic)) { return topicMetadata; @@ -252,8 +260,11 @@ public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { public Collection fetchTopicMetadata() { - final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics().toStruct(), ApiKeys.METADATA); - final MetadataResponse metadataResponse = new MetadataResponse(clientResponse.responseBody()); + final ClientResponse clientResponse = sendRequest(MetadataRequest.allTopics(), ApiKeys.METADATA); + if (!(clientResponse.responseBody() instanceof MetadataResponse)) { + throw new StreamsException("Inconsistent response type for internal topic metadata request. Expected MetadataResponse but received " + clientResponse.responseBody().getClass().getName()); + } + final MetadataResponse metadataResponse = (MetadataResponse) clientResponse.responseBody(); return metadataResponse.topicMetadata(); } From 3063b20f72e62ffb4ba4833f5c08a8dcf6df849a Mon Sep 17 00:00:00 2001 From: Hojjat Jafarpour Date: Fri, 6 Jan 2017 16:47:49 -0800 Subject: [PATCH 16/16] Made changes based on feedbacks. --- build.gradle | 1 - gradlew | 164 ------------------ gradlew.bat | 90 ---------- .../internals/InternalTopicManager.java | 39 +++-- .../internals/StreamPartitionAssignor.java | 4 +- .../internals/StreamsKafkaClient.java | 21 ++- .../kafka/test/MockInternalTopicManager.java | 13 -- 7 files changed, 33 insertions(+), 299 deletions(-) delete mode 100755 gradlew delete mode 100644 gradlew.bat diff --git a/build.gradle b/build.gradle index 31b9664874b2..97803e65df39 100644 --- a/build.gradle +++ b/build.gradle @@ -695,7 +695,6 @@ project(':streams') { compile project(':connect:json') // this dependency should be removed after we unify data API compile libs.slf4jApi compile libs.rocksDBJni - compile libs.jacksonDatabind testCompile project(':clients').sourceSets.test.output testCompile project(':core') diff --git a/gradlew b/gradlew deleted file mode 100755 index 27309d92314c..000000000000 --- a/gradlew +++ /dev/null @@ -1,164 +0,0 @@ -#!/usr/bin/env bash - -############################################################################## -## -## Gradle start up script for UN*X -## -############################################################################## - -# Attempt to set APP_HOME -# Resolve links: $0 may be a link -PRG="$0" -# Need this for relative symlinks. -while [ -h "$PRG" ] ; do - ls=`ls -ld "$PRG"` - link=`expr "$ls" : '.*-> \(.*\)$'` - if expr "$link" : '/.*' > /dev/null; then - PRG="$link" - else - PRG=`dirname "$PRG"`"/$link" - fi -done -SAVED="`pwd`" -cd "`dirname \"$PRG\"`/" >/dev/null -APP_HOME="`pwd -P`" -cd "$SAVED" >/dev/null - -APP_NAME="Gradle" -APP_BASE_NAME=`basename "$0"` - -# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. -DEFAULT_JVM_OPTS="" - -# Use the maximum available, or set MAX_FD != -1 to use that value. -MAX_FD="maximum" - -warn ( ) { - echo "$*" -} - -die ( ) { - echo - echo "$*" - echo - exit 1 -} - -# OS specific support (must be 'true' or 'false'). -cygwin=false -msys=false -darwin=false -nonstop=false -case "`uname`" in - CYGWIN* ) - cygwin=true - ;; - Darwin* ) - darwin=true - ;; - MINGW* ) - msys=true - ;; - NONSTOP* ) - nonstop=true - ;; -esac - -CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar - -# Determine the Java command to use to start the JVM. -if [ -n "$JAVA_HOME" ] ; then - if [ -x "$JAVA_HOME/jre/sh/java" ] ; then - # IBM's JDK on AIX uses strange locations for the executables - JAVACMD="$JAVA_HOME/jre/sh/java" - else - JAVACMD="$JAVA_HOME/bin/java" - fi - if [ ! -x "$JAVACMD" ] ; then - die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME - -Please set the JAVA_HOME variable in your environment to match the -location of your Java installation." - fi -else - JAVACMD="java" - which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. - -Please set the JAVA_HOME variable in your environment to match the -location of your Java installation." -fi - -# Increase the maximum file descriptors if we can. -if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then - MAX_FD_LIMIT=`ulimit -H -n` - if [ $? -eq 0 ] ; then - if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then - MAX_FD="$MAX_FD_LIMIT" - fi - ulimit -n $MAX_FD - if [ $? -ne 0 ] ; then - warn "Could not set maximum file descriptor limit: $MAX_FD" - fi - else - warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT" - fi -fi - -# For Darwin, add options to specify how the application appears in the dock -if $darwin; then - GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\"" -fi - -# For Cygwin, switch paths to Windows format before running java -if $cygwin ; then - APP_HOME=`cygpath --path --mixed "$APP_HOME"` - CLASSPATH=`cygpath --path --mixed "$CLASSPATH"` - JAVACMD=`cygpath --unix "$JAVACMD"` - - # We build the pattern for arguments to be converted via cygpath - ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null` - SEP="" - for dir in $ROOTDIRSRAW ; do - ROOTDIRS="$ROOTDIRS$SEP$dir" - SEP="|" - done - OURCYGPATTERN="(^($ROOTDIRS))" - # Add a user-defined pattern to the cygpath arguments - if [ "$GRADLE_CYGPATTERN" != "" ] ; then - OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)" - fi - # Now convert the arguments - kludge to limit ourselves to /bin/sh - i=0 - for arg in "$@" ; do - CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -` - CHECK2=`echo "$arg"|egrep -c "^-"` ### Determine if an option - - if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then ### Added a condition - eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"` - else - eval `echo args$i`="\"$arg\"" - fi - i=$((i+1)) - done - case $i in - (0) set -- ;; - (1) set -- "$args0" ;; - (2) set -- "$args0" "$args1" ;; - (3) set -- "$args0" "$args1" "$args2" ;; - (4) set -- "$args0" "$args1" "$args2" "$args3" ;; - (5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;; - (6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;; - (7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;; - (8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;; - (9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;; - esac -fi - -# Split up the JVM_OPTS And GRADLE_OPTS values into an array, following the shell quoting and substitution rules -function splitJvmOpts() { - JVM_OPTS=("$@") -} -eval splitJvmOpts $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS -JVM_OPTS[${#JVM_OPTS[*]}]="-Dorg.gradle.appname=$APP_BASE_NAME" - -exec "$JAVACMD" "${JVM_OPTS[@]}" -classpath "$CLASSPATH" org.gradle.wrapper.GradleWrapperMain "$@" diff --git a/gradlew.bat b/gradlew.bat deleted file mode 100644 index f6d5974e72fd..000000000000 --- a/gradlew.bat +++ /dev/null @@ -1,90 +0,0 @@ -@if "%DEBUG%" == "" @echo off -@rem ########################################################################## -@rem -@rem Gradle startup script for Windows -@rem -@rem ########################################################################## - -@rem Set local scope for the variables with windows NT shell -if "%OS%"=="Windows_NT" setlocal - -set DIRNAME=%~dp0 -if "%DIRNAME%" == "" set DIRNAME=. -set APP_BASE_NAME=%~n0 -set APP_HOME=%DIRNAME% - -@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. -set DEFAULT_JVM_OPTS= - -@rem Find java.exe -if defined JAVA_HOME goto findJavaFromJavaHome - -set JAVA_EXE=java.exe -%JAVA_EXE% -version >NUL 2>&1 -if "%ERRORLEVEL%" == "0" goto init - -echo. -echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. -echo. -echo Please set the JAVA_HOME variable in your environment to match the -echo location of your Java installation. - -goto fail - -:findJavaFromJavaHome -set JAVA_HOME=%JAVA_HOME:"=% -set JAVA_EXE=%JAVA_HOME%/bin/java.exe - -if exist "%JAVA_EXE%" goto init - -echo. -echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% -echo. -echo Please set the JAVA_HOME variable in your environment to match the -echo location of your Java installation. - -goto fail - -:init -@rem Get command-line arguments, handling Windows variants - -if not "%OS%" == "Windows_NT" goto win9xME_args -if "%@eval[2+2]" == "4" goto 4NT_args - -:win9xME_args -@rem Slurp the command line arguments. -set CMD_LINE_ARGS= -set _SKIP=2 - -:win9xME_args_slurp -if "x%~1" == "x" goto execute - -set CMD_LINE_ARGS=%* -goto execute - -:4NT_args -@rem Get arguments from the 4NT Shell from JP Software -set CMD_LINE_ARGS=%$ - -:execute -@rem Setup the command line - -set CLASSPATH=%APP_HOME%\gradle\wrapper\gradle-wrapper.jar - -@rem Execute Gradle -"%JAVA_EXE%" %DEFAULT_JVM_OPTS% %JAVA_OPTS% %GRADLE_OPTS% "-Dorg.gradle.appname=%APP_BASE_NAME%" -classpath "%CLASSPATH%" org.gradle.wrapper.GradleWrapperMain %CMD_LINE_ARGS% - -:end -@rem End local scope for the variables with windows NT shell -if "%ERRORLEVEL%"=="0" goto mainEnd - -:fail -rem Set variable GRADLE_EXIT_CONSOLE if you need the _script_ return code instead of -rem the _cmd.exe /c_ return code! -if not "" == "%GRADLE_EXIT_CONSOLE%" exit 1 -exit /b 1 - -:mainEnd -if "%OS%"=="Windows_NT" endlocal - -:omega diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index f67fa4434b42..5794e30a2c88 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -19,6 +19,8 @@ import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.streams.errors.StreamsException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Collection; @@ -28,6 +30,7 @@ public class InternalTopicManager { + private static final Logger log = LoggerFactory.getLogger(InternalTopicManager.class); public static final String CLEANUP_POLICY_PROP = "cleanup.policy"; public static final String RETENTION_MS = "retention.ms"; public static final Long WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS); @@ -44,36 +47,38 @@ public InternalTopicManager(final StreamsKafkaClient streamsKafkaClient, final i this.windowChangeLogAdditionalRetention = windowChangeLogAdditionalRetention; } - public void makeReady(final InternalTopicConfig topic, int numPartitions) { - Map topics = new HashMap<>(); - topics.put(topic, numPartitions); - makeReady(topics); - } - /** * Prepares the set of given internal topics. If the topic with the correct number of partitions exists ignores it. For the ones with different number of * partitions delete them and create new ones with correct number of partitons along with the non existing topics. - * @param topics + * @param topic */ - public void makeReady(final Map topics) { + public void makeReady(final InternalTopicConfig topic, int numPartitions) { - Collection topicMetadatas = streamsKafkaClient.fetchTopicMetadata(); - Map topicsToBeDeleted = getTopicsToBeDeleted(topics, topicMetadatas); - Map topicsToBeCreated = filterExistingTopics(topics, topicMetadatas); - topicsToBeCreated.putAll(topicsToBeDeleted); - streamsKafkaClient.deleteTopics(topicsToBeDeleted); + Map topics = new HashMap<>(); + topics.put(topic, numPartitions); for (int i = 0; i < MAX_TOPIC_READY_TRY; i++) { try { + Collection topicMetadatas = streamsKafkaClient.fetchTopicMetadata(); + Map topicsToBeDeleted = getTopicsToBeDeleted(topics, topicMetadatas); + Map topicsToBeCreated = filterExistingTopics(topics, topicMetadatas); + topicsToBeCreated.putAll(topicsToBeDeleted); + streamsKafkaClient.deleteTopics(topicsToBeDeleted); streamsKafkaClient.createTopics(topicsToBeCreated, replicationFactor, windowChangeLogAdditionalRetention); return; } catch (StreamsException ex) { + log.debug("Could not create internal topics: " + ex.getMessage()); + log.debug("Retry #" + i); } } throw new StreamsException("Could not create internal topics."); } - public void close() throws IOException { - streamsKafkaClient.close(); + public void close() { + try { + streamsKafkaClient.close(); + } catch (IOException e) { + log.warn("Could not close StreamsKafkaClient."); + } } /** @@ -83,7 +88,7 @@ public void close() throws IOException { * @param topicsMetadata * @return */ - public Map filterExistingTopics(final Map topicsPartitionsMap, Collection topicsMetadata) { + private Map filterExistingTopics(final Map topicsPartitionsMap, Collection topicsMetadata) { Map existingTopicNamesPartitions = getExistingTopicNamesPartitions(topicsMetadata); Map nonExistingTopics = new HashMap<>(); // Add the topics that don't exist to the nonExistingTopics. @@ -101,7 +106,7 @@ public Map filterExistingTopics(final Map getTopicsToBeDeleted(final Map topicsPartitionsMap, Collection topicsMetadata) { + private Map getTopicsToBeDeleted(final Map topicsPartitionsMap, Collection topicsMetadata) { Map existingTopicNamesPartitions = getExistingTopicNamesPartitions(topicsMetadata); Map deleteTopics = new HashMap<>(); // Add the topics that don't exist to the nonExistingTopics. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 7e9adf25a8eb..7b48a6fdb784 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -39,7 +39,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -262,7 +261,6 @@ public Subscription subscription(Set topics) { * * 3. within each client, tasks are assigned to consumer clients in round-robin manner. */ - @Override public Map assign(Cluster metadata, Map subscriptions) { @@ -747,7 +745,7 @@ public String toString() { } } - public void close() throws IOException { + public void close() { internalTopicManager.close(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index 2b78d1f4f800..132c1afe0839 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -190,24 +190,23 @@ private void deleteTopics(final Set topics) { private ClientResponse sendRequest(final AbstractRequest request, final ApiKeys apiKeys) { String brokerId = null; - final SystemTime systemTime = new SystemTime(); final Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - metadata.update(Cluster.bootstrap(addresses), systemTime.milliseconds()); + metadata.update(Cluster.bootstrap(addresses), Time.SYSTEM.milliseconds()); final List nodes = metadata.fetch().nodes(); - final long readyTimeout = systemTime.milliseconds() + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG); + final long readyTimeout = Time.SYSTEM.milliseconds() + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG); boolean foundNode = false; - while (!foundNode && (systemTime.milliseconds() < readyTimeout)) { + while (!foundNode && (Time.SYSTEM.milliseconds() < readyTimeout)) { for (Node node: nodes) { - if (kafkaClient.ready(node, systemTime.milliseconds())) { + if (kafkaClient.ready(node, Time.SYSTEM.milliseconds())) { brokerId = Integer.toString(node.id()); foundNode = true; break; } } - kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); + kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), Time.SYSTEM.milliseconds()); } if (brokerId == null) { throw new StreamsException("Could not find any available broker."); @@ -215,14 +214,14 @@ private ClientResponse sendRequest(final AbstractRequest request, final ApiKeys final RequestHeader requestHeader = kafkaClient.nextRequestHeader(apiKeys); - final ClientRequest clientRequest = new ClientRequest(brokerId, systemTime.milliseconds(), true, requestHeader, request, null); + final ClientRequest clientRequest = new ClientRequest(brokerId, Time.SYSTEM.milliseconds(), true, requestHeader, request, null); - kafkaClient.send(clientRequest, systemTime.milliseconds()); + kafkaClient.send(clientRequest, Time.SYSTEM.milliseconds()); - final long responseTimeout = systemTime.milliseconds() + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG); + final long responseTimeout = Time.SYSTEM.milliseconds() + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG); // Poll for the response. - while (systemTime.milliseconds() < responseTimeout) { - List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), systemTime.milliseconds()); + while (Time.SYSTEM.milliseconds() < responseTimeout) { + List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), Time.SYSTEM.milliseconds()); if (!responseList.isEmpty()) { if (responseList.size() > 1) { throw new StreamsException("Sent one request but received multiple or no responses."); diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java index db40059b1586..af268801dbba 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java +++ b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java @@ -52,17 +52,4 @@ public void makeReady(InternalTopicConfig topic, int numPartitions) { restoreConsumer.updatePartitions(topic.name(), partitions); } - @Override - public void makeReady(final Map topics) { - for (InternalTopicConfig topic: topics.keySet()) { - int numPartitions = topics.get(topic); - readyTopics.put(topic.name(), numPartitions); - List partitions = new ArrayList<>(); - for (int i = 0; i < numPartitions; i++) { - partitions.add(new PartitionInfo(topic.name(), i, null, null, null)); - } - restoreConsumer.updatePartitions(topic.name(), partitions); - } - - } } \ No newline at end of file