PageRenderTime 39ms CodeModel.GetById 28ms RepoModel.GetById 9ms app.codeStats 0ms

/src/main/java/com/sohu/jafka/consumer/ZookeeperConsumerConnector.java

https://github.com/mailmahee/jafka
Java | 812 lines | 541 code | 74 blank | 197 comment | 66 complexity | 5df3a6d7c84ebf81bcb0c7f612217405 MD5 | raw file
Possible License(s): Apache-2.0
  1. /**
  2. * Licensed to the Apache Software Foundation (ASF) under one or more
  3. * contributor license agreements. See the NOTICE file distributed with
  4. * this work for additional information regarding copyright ownership.
  5. * The ASF licenses this file to You under the Apache License, Version 2.0
  6. * (the "License"); you may not use this file except in compliance with
  7. * the License. You may obtain a copy of the License at
  8. *
  9. * http://www.apache.org/licenses/LICENSE-2.0
  10. *
  11. * Unless required by applicable law or agreed to in writing, software
  12. * distributed under the License is distributed on an "AS IS" BASIS,
  13. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  14. * See the License for the specific language governing permissions and
  15. * limitations under the License.
  16. */
  17. package com.sohu.jafka.consumer;
  18. import java.io.Closeable;
  19. import java.io.IOException;
  20. import java.net.InetAddress;
  21. import java.net.UnknownHostException;
  22. import java.util.ArrayList;
  23. import java.util.Collection;
  24. import java.util.HashMap;
  25. import java.util.List;
  26. import java.util.Map;
  27. import java.util.Map.Entry;
  28. import java.util.Set;
  29. import java.util.UUID;
  30. import java.util.concurrent.BlockingQueue;
  31. import java.util.concurrent.CountDownLatch;
  32. import java.util.concurrent.LinkedBlockingQueue;
  33. import java.util.concurrent.TimeUnit;
  34. import java.util.concurrent.atomic.AtomicBoolean;
  35. import java.util.concurrent.atomic.AtomicLong;
  36. import java.util.concurrent.locks.Condition;
  37. import java.util.concurrent.locks.ReentrantLock;
  38. import org.apache.log4j.Logger;
  39. import org.apache.zookeeper.Watcher.Event.KeeperState;
  40. import com.github.zkclient.IZkChildListener;
  41. import com.github.zkclient.IZkStateListener;
  42. import com.github.zkclient.ZkClient;
  43. import com.github.zkclient.exception.ZkNodeExistsException;
  44. import com.sohu.jafka.api.OffsetRequest;
  45. import com.sohu.jafka.cluster.Broker;
  46. import com.sohu.jafka.cluster.Cluster;
  47. import com.sohu.jafka.cluster.Partition;
  48. import com.sohu.jafka.common.ConsumerRebalanceFailedException;
  49. import com.sohu.jafka.common.InvalidConfigException;
  50. import com.sohu.jafka.producer.serializer.Decoder;
  51. import com.sohu.jafka.utils.Closer;
  52. import com.sohu.jafka.utils.KV.StringTuple;
  53. import com.sohu.jafka.utils.Pool;
  54. import com.sohu.jafka.utils.Scheduler;
  55. import com.sohu.jafka.utils.zookeeper.ZkGroupDirs;
  56. import com.sohu.jafka.utils.zookeeper.ZkGroupTopicDirs;
  57. import com.sohu.jafka.utils.zookeeper.ZkUtils;
  58. import static java.lang.String.format;
  59. /**
  60. * This class handles the consumers interaction with zookeeper
  61. *
  62. * Directories:
  63. * <p>
  64. * <b>1. Consumer id registry:</b>
  65. *
  66. * <pre>
  67. * /consumers/[group_id]/ids[consumer_id] -> topic1,...topicN
  68. * </pre>
  69. *
  70. * A consumer has a unique consumer id within a consumer group. A consumer registers its id as
  71. * an ephemeral znode and puts all topics that it subscribes to as the value of the znode. The
  72. * znode is deleted when the client is gone. A consumer subscribes to event changes of the
  73. * consumer id registry within its group.
  74. * </p>
  75. * <p>
  76. * The consumer id is picked up from configuration, instead of the sequential id assigned by
  77. * ZK. Generated sequential ids are hard to recover during temporary connection loss to ZK,
  78. * since it's difficult for the client to figure out whether the creation of a sequential znode
  79. * has succeeded or not. More details can be found at
  80. * (http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling)
  81. * </p>
  82. * <p>
  83. * <b>2. Broker node registry:</b>
  84. *
  85. * <pre>
  86. * /brokers/[0...N] --> { "host" : "host:port",
  87. * "topics" : {"topic1": ["partition1" ... "partitionN"], ...,
  88. * "topicN": ["partition1" ... "partitionN"] } }
  89. * </pre>
  90. *
  91. * This is a list of all present broker brokers. A unique logical node id is configured on each
  92. * broker node. A broker node registers itself on start-up and creates a znode with the logical
  93. * node id under /brokers.
  94. *
  95. * The value of the znode is a JSON String that contains
  96. *
  97. * <pre>
  98. * (1) the host name and the port the broker is listening to,
  99. * (2) a list of topics that the broker serves,
  100. * (3) a list of logical partitions assigned to each topic on the broker.
  101. * </pre>
  102. *
  103. * A consumer subscribes to event changes of the broker node registry.
  104. * </p>
  105. *
  106. * <p>
  107. * <b>3. Partition owner registry:</b>
  108. *
  109. * <pre>
  110. * /consumers/[group_id]/owner/[topic]/[broker_id-partition_id] --> consumer_node_id
  111. * </pre>
  112. *
  113. * This stores the mapping before broker partitions and consumers. Each partition is owned by a
  114. * unique consumer within a consumer group. The mapping is reestablished after each
  115. * rebalancing.
  116. * </p>
  117. *
  118. * <p>
  119. * <b>4. Consumer offset tracking:</b>
  120. *
  121. * <pre>
  122. * /consumers/[group_id]/offsets/[topic]/[broker_id-partition_id] --> offset_counter_value
  123. * </pre>
  124. *
  125. * Each consumer tracks the offset of the latest message consumed for each partition.
  126. * </p>
  127. *
  128. * @author adyliu (imxylz@gmail.com)
  129. * @since 1.0
  130. */
  131. public class ZookeeperConsumerConnector implements ConsumerConnector {
  132. public static final FetchedDataChunk SHUTDOWN_COMMAND = new FetchedDataChunk(null, null, -1);
  133. private final Logger logger = Logger.getLogger(ZookeeperConsumerConnector.class);
  134. private final AtomicBoolean isShuttingDown = new AtomicBoolean(false);
  135. private final Object rebalanceLock = new Object();
  136. private Fetcher fetcher;
  137. private ZkClient zkClient;
  138. private Pool<String, Pool<Partition, PartitionTopicInfo>> topicRegistry;
  139. //
  140. private final Pool<StringTuple, BlockingQueue<FetchedDataChunk>> queues;
  141. private final Scheduler scheduler = new Scheduler(1, "consumer-autocommit-", false);
  142. final ConsumerConfig config;
  143. final boolean enableFetcher;
  144. //cache for shutdown
  145. private List<ZKRebalancerListener<?>> rebalancerListeners = new ArrayList<ZKRebalancerListener<?>>();
  146. public ZookeeperConsumerConnector(ConsumerConfig config) {
  147. this(config, true);
  148. }
  149. public ZookeeperConsumerConnector(ConsumerConfig config, boolean enableFetcher) {
  150. this.config = config;
  151. this.enableFetcher = enableFetcher;
  152. //
  153. this.topicRegistry = new Pool<String, Pool<Partition, PartitionTopicInfo>>();
  154. this.queues = new Pool<StringTuple, BlockingQueue<FetchedDataChunk>>();
  155. //
  156. connectZk();
  157. createFetcher();
  158. if (this.config.isAutoCommit()) {
  159. logger.info("starting auto committer every " + config.getAutoCommitIntervalMs() + " ms");
  160. scheduler.scheduleWithRate(new AutoCommitTask(), config.getAutoCommitIntervalMs(),
  161. config.getAutoCommitIntervalMs());
  162. }
  163. }
  164. /**
  165. *
  166. */
  167. private void createFetcher() {
  168. if (enableFetcher) {
  169. this.fetcher = new Fetcher(config, zkClient);
  170. }
  171. }
  172. class AutoCommitTask implements Runnable {
  173. public void run() {
  174. try {
  175. commitOffsets();
  176. } catch (Throwable e) {
  177. logger.error("exception during autoCommit: ", e);
  178. }
  179. }
  180. }
  181. public <T> Map<String, List<MessageStream<T>>> createMessageStreams(Map<String, Integer> topicCountMap,
  182. Decoder<T> decoder) {
  183. return consume(topicCountMap, decoder);
  184. }
  185. private <T> Map<String, List<MessageStream<T>>> consume(Map<String, Integer> topicCountMap, Decoder<T> decoder) {
  186. if (topicCountMap == null) {
  187. throw new IllegalArgumentException("topicCountMap is null");
  188. }
  189. //
  190. ZkGroupDirs dirs = new ZkGroupDirs(config.getGroupId());
  191. Map<String, List<MessageStream<T>>> ret = new HashMap<String, List<MessageStream<T>>>();
  192. String consumerUuid = config.getConsumerId();
  193. if (consumerUuid == null) {
  194. consumerUuid = generateConsumerId();
  195. }
  196. logger.info(format("create message stream by consumerid [%s] with groupid [%s]", consumerUuid,
  197. config.getGroupId()));
  198. //
  199. //consumerIdString => groupid_consumerid
  200. final String consumerIdString = config.getGroupId() + "_" + consumerUuid;
  201. final TopicCount topicCount = new TopicCount(consumerIdString, topicCountMap);
  202. for (Map.Entry<String, Set<String>> e : topicCount.getConsumerThreadIdsPerTopic().entrySet()) {
  203. final String topic = e.getKey();
  204. final Set<String> threadIdSet = e.getValue();
  205. final List<MessageStream<T>> streamList = new ArrayList<MessageStream<T>>();
  206. for (String threadId : threadIdSet) {
  207. LinkedBlockingQueue<FetchedDataChunk> stream = new LinkedBlockingQueue<FetchedDataChunk>(
  208. config.getMaxQueuedChunks());
  209. queues.put(new StringTuple(topic, threadId), stream);
  210. streamList.add(new MessageStream<T>(topic, stream, config.getConsumerTimeoutMs(), decoder));
  211. }
  212. ret.put(topic, streamList);
  213. logger.debug("adding topic " + topic + " and stream to map.");
  214. }
  215. //
  216. //listener to consumer and partition changes
  217. ZKRebalancerListener<T> loadBalancerListener = new ZKRebalancerListener<T>(config.getGroupId(),
  218. consumerIdString, ret);
  219. this.rebalancerListeners.add(loadBalancerListener);
  220. loadBalancerListener.start();
  221. registerConsumerInZK(dirs, consumerIdString, topicCount);
  222. //
  223. //register listener for session expired event
  224. zkClient.subscribeStateChanges(new ZKSessionExpireListener<T>(dirs, consumerIdString, topicCount,
  225. loadBalancerListener));
  226. zkClient.subscribeChildChanges(dirs.consumerRegistryDir, loadBalancerListener);
  227. //
  228. for (String topic : ret.keySet()) {
  229. //register on broker partition path changes
  230. final String partitionPath = ZkUtils.BrokerTopicsPath + "/" + topic;
  231. zkClient.subscribeChildChanges(partitionPath, loadBalancerListener);
  232. }
  233. //explicitly grigger load balancing for this consumer
  234. loadBalancerListener.syncedRebalance();
  235. return ret;
  236. }
  237. /**
  238. * generate random consumerid ( hostname-currenttime-uuid.sub(8) )
  239. *
  240. * @return random consumerid
  241. */
  242. private String generateConsumerId() {
  243. UUID uuid = UUID.randomUUID();
  244. try {
  245. return format("%s-%d-%s", InetAddress.getLocalHost().getHostName(), //
  246. System.currentTimeMillis(),//
  247. Long.toHexString(uuid.getMostSignificantBits()).substring(0, 8));
  248. } catch (UnknownHostException e) {
  249. throw new IllegalArgumentException(
  250. "can not generate consume id by auto, set the 'consumerid' parameter to fix this");
  251. }
  252. }
  253. public void commitOffsets() {
  254. if (zkClient == null) {
  255. logger.error("zk client is null. Cannot commit offsets");
  256. return;
  257. }
  258. for (Entry<String, Pool<Partition, PartitionTopicInfo>> e : topicRegistry.entrySet()) {
  259. ZkGroupTopicDirs topicDirs = new ZkGroupTopicDirs(config.getGroupId(), e.getKey());
  260. //
  261. for (PartitionTopicInfo info : e.getValue().values()) {
  262. final long lastChanged = info.getConsumedOffsetChanged().get();
  263. if (lastChanged == 0) {
  264. logger.trace("consume offset not changed");
  265. continue;
  266. }
  267. final long newOffset = info.getConsumedOffset();
  268. //path: /consumers/<group>/offsets/<topic>/<brokerid-partition>
  269. final String path = topicDirs.consumerOffsetDir + "/" + info.partition.getName();
  270. try {
  271. ZkUtils.updatePersistentPath(zkClient, path, "" + newOffset);
  272. } catch (Throwable t) {
  273. logger.warn("exception during commitOffsets, path=" + path + ",offset=" + newOffset, t);
  274. } finally {
  275. info.resetComsumedOffsetChanged(lastChanged);
  276. if (logger.isDebugEnabled()) {
  277. logger.debug("Committed [" + path + "] for topic " + info);
  278. }
  279. }
  280. }
  281. //
  282. }
  283. }
  284. public void close() throws IOException {
  285. if (isShuttingDown.compareAndSet(false, true)) {
  286. logger.info("ZkConsumerConnector shutting down");
  287. try {
  288. scheduler.shutdown();
  289. if (fetcher != null) {
  290. fetcher.stopConnectionsToAllBrokers();
  291. }
  292. sendShutdownToAllQueues();
  293. if (config.isAutoCommit()) {
  294. commitOffsets();
  295. }
  296. //waiting rebalance listener to closed and then shutdown the zkclient
  297. for (ZKRebalancerListener<?> listener : this.rebalancerListeners) {
  298. Closer.closeQuietly(listener);
  299. }
  300. if (this.zkClient != null) {
  301. this.zkClient.close();
  302. zkClient = null;
  303. }
  304. } catch (Exception e) {
  305. logger.fatal("error during consumer connector shutdown", e);
  306. }
  307. logger.info("ZkConsumerConnector shut down completed");
  308. }
  309. }
  310. private void sendShutdownToAllQueues() {
  311. for (BlockingQueue<FetchedDataChunk> queue : queues.values()) {
  312. queue.clear();
  313. try {
  314. queue.put(SHUTDOWN_COMMAND);
  315. } catch (InterruptedException e) {
  316. logger.warn(e.getMessage(),e);
  317. }
  318. }
  319. }
  320. private void connectZk() {
  321. logger.info("Connecting to zookeeper instance at " + config.getZkConnect());
  322. this.zkClient = new ZkClient(config.getZkConnect(), config.getZkSessionTimeoutMs(),
  323. config.getZkConnectionTimeoutMs());
  324. logger.info("Connected to zookeeper at " + config.getZkConnect());
  325. }
  326. class ZKRebalancerListener<T> implements IZkChildListener, Runnable, Closeable {
  327. final String group;
  328. final String consumerIdString;
  329. Map<String, List<MessageStream<T>>> messagesStreams;
  330. //
  331. private boolean isWatcherTriggered = false;
  332. private final ReentrantLock lock = new ReentrantLock();
  333. private final Condition cond = lock.newCondition();
  334. private final Thread watcherExecutorThread;
  335. private CountDownLatch shutDownLatch = new CountDownLatch(1);
  336. public ZKRebalancerListener(String group, String consumerIdString,
  337. Map<String, List<MessageStream<T>>> messagesStreams) {
  338. super();
  339. this.group = group;
  340. this.consumerIdString = consumerIdString;
  341. this.messagesStreams = messagesStreams;
  342. //
  343. this.watcherExecutorThread = new Thread(this, consumerIdString + "_watcher_executor");
  344. }
  345. public void start() {
  346. this.watcherExecutorThread.start();
  347. }
  348. public void handleChildChange(String parentPath, List<String> currentChilds) throws Exception {
  349. lock.lock();
  350. try {
  351. isWatcherTriggered = true;
  352. cond.signalAll();
  353. } finally {
  354. lock.unlock();
  355. }
  356. }
  357. @Override
  358. public void close() {
  359. lock.lock();
  360. try {
  361. isWatcherTriggered = false;
  362. cond.signalAll();
  363. } finally {
  364. lock.unlock();
  365. }
  366. try {
  367. shutDownLatch.await(5, TimeUnit.SECONDS);
  368. } catch (InterruptedException e) {
  369. //ignore
  370. }
  371. }
  372. public void run() {
  373. logger.info("starting watcher executor thread for consumer " + consumerIdString);
  374. boolean doRebalance = false;
  375. while (!isShuttingDown.get()) {
  376. try {
  377. lock.lock();
  378. try {
  379. if (!isWatcherTriggered) {
  380. cond.await(1000, TimeUnit.MILLISECONDS);//wake up periodically so that it can check the shutdown flag
  381. }
  382. } finally {
  383. doRebalance = isWatcherTriggered;
  384. isWatcherTriggered = false;
  385. lock.unlock();
  386. }
  387. if (doRebalance) {
  388. syncedRebalance();
  389. }
  390. } catch (Throwable t) {
  391. logger.error("error during syncedRebalance", t);
  392. }
  393. }
  394. //
  395. logger.info("stopped thread " + watcherExecutorThread.getName());
  396. shutDownLatch.countDown();
  397. }
  398. public void syncedRebalance() {
  399. synchronized (rebalanceLock) {
  400. for (int i = 0; i < config.getMaxRebalanceRetries(); i++) {
  401. if (isShuttingDown.get()) {//do nothing while shutting down
  402. return;
  403. }
  404. logger.info(format("[%s] rebalancing starting. try #%d", consumerIdString, i));
  405. final long start = System.currentTimeMillis();
  406. boolean done = false;
  407. Cluster cluster = ZkUtils.getCluster(zkClient);
  408. try {
  409. done = rebalance(cluster);
  410. } catch (Exception e) {
  411. /**
  412. * occasionally, we may hit a ZK exception because the ZK state is
  413. * changing while we are iterating. For example, a ZK node can
  414. * disappear between the time we get all children and the time we try
  415. * to get the value of a child. Just let this go since another
  416. * rebalance will be triggered.
  417. **/
  418. logger.info("exception during rebalance ", e);
  419. }
  420. logger.info(format("[%s] rebalanced %s. try #%d, cost %d ms",//
  421. consumerIdString, done ? "OK" : "FAILED",//
  422. i, System.currentTimeMillis() - start));
  423. //
  424. if (done) {
  425. return;
  426. } else {
  427. /* Here the cache is at a risk of being stale. To take future rebalancing decisions correctly, we should
  428. * clear the cache */
  429. logger.warn("Rebalancing attempt failed. Clearing the cache before the next rebalancing operation is triggered");
  430. }
  431. //
  432. closeFetchersForQueues(cluster, messagesStreams, queues.values());
  433. try {
  434. Thread.sleep(config.getRebalanceBackoffMs());
  435. } catch (InterruptedException e) {
  436. logger.warn(e.getMessage());
  437. }
  438. }
  439. }
  440. throw new ConsumerRebalanceFailedException(
  441. consumerIdString + " can't rebalance after " + config.getMaxRebalanceRetries() + " retries");
  442. }
  443. private boolean rebalance(Cluster cluster) {
  444. // map for current consumer: topic->[groupid-consumer-0,groupid-consumer-1,...,groupid-consumer-N]
  445. Map<String, Set<String>> myTopicThreadIdsMap = ZkUtils.getTopicCount(zkClient, group, consumerIdString)
  446. .getConsumerThreadIdsPerTopic();
  447. // map for all consumers in this group: topic->[groupid-consumer1-0,...,groupid-consumerX-N]
  448. Map<String, List<String>> consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group);
  449. // map for all broker-partitions for the topics in this consumerid: topic->[brokerid0-partition0,...,brokeridN-partitionN]
  450. Map<String, List<String>> brokerPartitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient,
  451. myTopicThreadIdsMap.keySet());
  452. /**
  453. * fetchers must be stopped to avoid data duplication, since if the current
  454. * rebalancing attempt fails, the partitions that are released could be owned by
  455. * another consumer. But if we don't stop the fetchers first, this consumer would
  456. * continue returning data for released partitions in parallel. So, not stopping
  457. * the fetchers leads to duplicate data.
  458. */
  459. closeFetchers(cluster, messagesStreams, myTopicThreadIdsMap);
  460. releasePartitionOwnership(topicRegistry);
  461. //
  462. Map<StringTuple, String> partitionOwnershipDecision = new HashMap<StringTuple, String>();
  463. Pool<String, Pool<Partition, PartitionTopicInfo>> currentTopicRegistry = new Pool<String, Pool<Partition, PartitionTopicInfo>>();
  464. for (Map.Entry<String, Set<String>> e : myTopicThreadIdsMap.entrySet()) {
  465. final String topic = e.getKey();
  466. currentTopicRegistry.put(topic, new Pool<Partition, PartitionTopicInfo>());
  467. //
  468. ZkGroupTopicDirs topicDirs = new ZkGroupTopicDirs(group, topic);
  469. List<String> curConsumers = consumersPerTopicMap.get(topic);
  470. List<String> curBrokerPartitions = brokerPartitionsPerTopicMap.get(topic);
  471. final int nPartsPerConsumer = curBrokerPartitions.size() / curConsumers.size();
  472. final int nConsumersWithExtraPart = curBrokerPartitions.size() % curConsumers.size();
  473. logger.info("Consumer " + consumerIdString + " rebalancing the following partitions:\n "//
  474. + curBrokerPartitions + "\nfor topic " + topic + " with consumers:\n " + curConsumers);
  475. if (logger.isDebugEnabled()) {
  476. StringBuilder buf = new StringBuilder(1024);
  477. buf.append("[").append(topic).append("] preassigning details:");
  478. for (int i = 0; i < curConsumers.size(); i++) {
  479. final int startPart = nPartsPerConsumer * i + Math.min(i, nConsumersWithExtraPart);
  480. final int nParts = nPartsPerConsumer + ((i + 1 > nConsumersWithExtraPart) ? 0 : 1);
  481. if (nParts > 0) {
  482. for (int m = startPart; m < startPart + nParts; m++) {
  483. buf.append("\n ").append(curConsumers.get(i)).append(" ==> ")
  484. .append(curBrokerPartitions.get(m));
  485. }
  486. }
  487. }
  488. logger.debug(buf.toString());
  489. }
  490. //consumerThreadId=> groupid_consumerid-index (index from count)
  491. for (String consumerThreadId : e.getValue()) {
  492. final int myConsumerPosition = curConsumers.indexOf(consumerThreadId);
  493. assert (myConsumerPosition >= 0);
  494. final int startPart = nPartsPerConsumer * myConsumerPosition + Math.min(myConsumerPosition,
  495. nConsumersWithExtraPart);
  496. final int nParts = nPartsPerConsumer + ((myConsumerPosition + 1 > nConsumersWithExtraPart) ? 0 : 1);
  497. /**
  498. * Range-partition the sorted partitions to consumers for better locality.
  499. * The first few consumers pick up an extra partition, if any.
  500. */
  501. if (nParts <= 0) {
  502. logger.warn("No broker partitions consumed by consumer thread " + consumerThreadId + " for topic " + topic);
  503. logger.warn("Check the consumer threads or the brokers for topic " + topic);
  504. } else {
  505. for (int i = startPart; i < startPart + nParts; i++) {
  506. String brokerPartition = curBrokerPartitions.get(i);
  507. logger.info("[" + consumerThreadId + "] ==> " + brokerPartition + " claimming");
  508. addPartitionTopicInfo(currentTopicRegistry, topicDirs, brokerPartition, topic,
  509. consumerThreadId);
  510. // record the partition ownership decision
  511. partitionOwnershipDecision.put(new StringTuple(topic, brokerPartition), consumerThreadId);
  512. }
  513. }
  514. }
  515. }
  516. //
  517. /**
  518. * move the partition ownership here, since that can be used to indicate a truly
  519. * successful rebalancing attempt A rebalancing attempt is completed successfully
  520. * only after the fetchers have been started correctly
  521. */
  522. if (reflectPartitionOwnershipDecision(partitionOwnershipDecision)) {
  523. logger.debug("Updating the cache");
  524. logger.debug("Partitions per topic cache " + brokerPartitionsPerTopicMap);
  525. logger.debug("Consumers per topic cache " + consumersPerTopicMap);
  526. topicRegistry = currentTopicRegistry;
  527. updateFetcher(cluster, messagesStreams);
  528. return true;
  529. } else {
  530. return false;
  531. }
  532. ////////////////////////////
  533. }
  534. private void updateFetcher(Cluster cluster, Map<String, List<MessageStream<T>>> messagesStreams2) {
  535. if (fetcher != null) {
  536. List<PartitionTopicInfo> allPartitionInfos = new ArrayList<PartitionTopicInfo>();
  537. for (Pool<Partition, PartitionTopicInfo> p : topicRegistry.values()) {
  538. allPartitionInfos.addAll(p.values());
  539. }
  540. fetcher.startConnections(allPartitionInfos, cluster, messagesStreams2);
  541. }
  542. }
  543. private boolean reflectPartitionOwnershipDecision(Map<StringTuple, String> partitionOwnershipDecision) {
  544. final List<StringTuple> successfullyOwnerdPartitions = new ArrayList<StringTuple>();
  545. int hasPartitionOwnershipFailed = 0;
  546. for (Map.Entry<StringTuple, String> e : partitionOwnershipDecision.entrySet()) {
  547. final String topic = e.getKey().k;
  548. final String brokerPartition = e.getKey().v;
  549. final String consumerThreadId = e.getValue();
  550. final ZkGroupTopicDirs topicDirs = new ZkGroupTopicDirs(group, topic);
  551. final String partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + brokerPartition;
  552. try {
  553. ZkUtils.createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId);
  554. successfullyOwnerdPartitions.add(new StringTuple(topic, brokerPartition));
  555. } catch (ZkNodeExistsException e2) {
  556. logger.warn(format("[%s] waiting [%s] to release => %s",//
  557. consumerThreadId,//
  558. ZkUtils.readDataMaybeNull(zkClient, partitionOwnerPath),//
  559. brokerPartition));
  560. hasPartitionOwnershipFailed++;
  561. }
  562. }
  563. //
  564. if (hasPartitionOwnershipFailed > 0) {
  565. for (StringTuple topicAndPartition : successfullyOwnerdPartitions) {
  566. deletePartitionOwnershipFromZK(topicAndPartition.k, topicAndPartition.v);
  567. }
  568. return false;
  569. }
  570. return true;
  571. }
  572. /**
  573. * @param currentTopicRegistry
  574. * @param topicDirs
  575. * @param brokerPartition broker-partition format
  576. * @param topic
  577. * @param consumerThreadId
  578. */
  579. private void addPartitionTopicInfo(Pool<String, Pool<Partition, PartitionTopicInfo>> currentTopicRegistry,
  580. ZkGroupTopicDirs topicDirs, String brokerPartition, String topic, String consumerThreadId) {
  581. Partition partition = Partition.parse(brokerPartition);
  582. Pool<Partition, PartitionTopicInfo> partTopicInfoMap = currentTopicRegistry.get(topic);
  583. final String znode = topicDirs.consumerOffsetDir + "/" + partition.getName();
  584. String offsetString = ZkUtils.readDataMaybeNull(zkClient, znode);
  585. // If first time starting a consumer, set the initial offset based on the config
  586. long offset = 0L;
  587. if (offsetString == null) {
  588. if (OffsetRequest.SMALLES_TIME_STRING.equals(config.getAutoOffsetReset())) {
  589. offset = earliestOrLatestOffset(topic, partition.brokerId, partition.partId,
  590. OffsetRequest.EARLIES_TTIME);
  591. } else if (OffsetRequest.LARGEST_TIME_STRING.equals(config.getAutoOffsetReset())) {
  592. offset = earliestOrLatestOffset(topic, partition.brokerId, partition.partId,
  593. OffsetRequest.LATES_TTIME);
  594. } else {
  595. throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig");
  596. }
  597. } else {
  598. offset = Long.parseLong(offsetString);
  599. }
  600. BlockingQueue<FetchedDataChunk> queue = queues.get(new StringTuple(topic, consumerThreadId));
  601. AtomicLong consumedOffset = new AtomicLong(offset);
  602. AtomicLong fetchedOffset = new AtomicLong(offset);
  603. PartitionTopicInfo partTopicInfo = new PartitionTopicInfo(topic,//
  604. partition,//
  605. queue,//
  606. consumedOffset,//
  607. fetchedOffset);//
  608. partTopicInfoMap.put(partition, partTopicInfo);
  609. logger.debug(partTopicInfo + " selected new offset " + offset);
  610. }
  611. private long earliestOrLatestOffset(String topic, int brokerId, int partitionId, long earliestOrLatest) {
  612. SimpleConsumer simpleConsumer = null;
  613. long producedOffset = -1;
  614. try {
  615. Cluster cluster = ZkUtils.getCluster(zkClient);
  616. Broker broker = cluster.getBroker(brokerId);
  617. if (broker == null) {
  618. throw new IllegalStateException(
  619. "Broker " + brokerId + " is unavailable. Cannot issue getOffsetsBefore request");
  620. }
  621. //
  622. //using default value???
  623. simpleConsumer = new SimpleConsumer(broker.host, broker.port, config.getSocketTimeoutMs(),
  624. config.getSocketBufferSize());
  625. long[] offsets = simpleConsumer.getOffsetsBefore(topic, partitionId, earliestOrLatest, 1);
  626. //FIXME: what's this!!!
  627. if (offsets.length > 0) {
  628. producedOffset = offsets[0];
  629. }
  630. } catch (Exception e) {
  631. logger.error("error in earliestOrLatestOffset() ", e);
  632. } finally {
  633. if (simpleConsumer != null) {
  634. Closer.closeQuietly(simpleConsumer);
  635. }
  636. }
  637. return producedOffset;
  638. }
  639. private void releasePartitionOwnership(Pool<String, Pool<Partition, PartitionTopicInfo>> localTopicRegistry) {
  640. logger.info("Releasing partition ownership => " + localTopicRegistry.values());
  641. for (Map.Entry<String, Pool<Partition, PartitionTopicInfo>> e : localTopicRegistry.entrySet()) {
  642. for (Partition partition : e.getValue().keySet()) {
  643. deletePartitionOwnershipFromZK(e.getKey(), partition);
  644. }
  645. }
  646. localTopicRegistry.clear();//clear all
  647. }
  648. private void deletePartitionOwnershipFromZK(String topic, String partitionStr) {
  649. ZkGroupTopicDirs topicDirs = new ZkGroupTopicDirs(group, topic);
  650. final String znode = topicDirs.consumerOwnerDir + "/" + partitionStr;
  651. ZkUtils.deletePath(zkClient, znode);
  652. logger.debug("Consumer [" + consumerIdString + "] released " + znode);
  653. }
  654. private void deletePartitionOwnershipFromZK(String topic, Partition partition) {
  655. this.deletePartitionOwnershipFromZK(topic, partition.toString());
  656. }
  657. /**
  658. * @param cluster
  659. * @param messagesStreams2
  660. * @param myTopicThreadIdsMap
  661. */
  662. private void closeFetchers(Cluster cluster, Map<String, List<MessageStream<T>>> messagesStreams2,
  663. Map<String, Set<String>> myTopicThreadIdsMap) {
  664. // topicRegistry.values()
  665. List<BlockingQueue<FetchedDataChunk>> queuesToBeCleared = new ArrayList<BlockingQueue<FetchedDataChunk>>();
  666. for (Map.Entry<StringTuple, BlockingQueue<FetchedDataChunk>> e : queues.entrySet()) {
  667. if (myTopicThreadIdsMap.containsKey(e.getKey().k)) {
  668. queuesToBeCleared.add(e.getValue());
  669. }
  670. }
  671. closeFetchersForQueues(cluster, messagesStreams2, queuesToBeCleared);
  672. }
  673. private void closeFetchersForQueues(Cluster cluster, Map<String, List<MessageStream<T>>> messageStreams,
  674. Collection<BlockingQueue<FetchedDataChunk>> queuesToBeCleared) {
  675. if (fetcher == null) {
  676. return;
  677. }
  678. fetcher.stopConnectionsToAllBrokers();
  679. fetcher.clearFetcherQueues(queuesToBeCleared, messageStreams.values());
  680. }
  681. private void resetState() {
  682. topicRegistry.clear();
  683. }
  684. ////////////////////////////////////////////////////////////
  685. }
  686. class ZKSessionExpireListener<T> implements IZkStateListener {
  687. private final ZkGroupDirs zkGroupDirs;
  688. private String consumerIdString;
  689. private TopicCount topicCount;
  690. private ZKRebalancerListener<T> loadRebalancerListener;
  691. public ZKSessionExpireListener(ZkGroupDirs zkGroupDirs, String consumerIdString, TopicCount topicCount,
  692. ZKRebalancerListener<T> loadRebalancerListener) {
  693. super();
  694. this.zkGroupDirs = zkGroupDirs;
  695. this.consumerIdString = consumerIdString;
  696. this.topicCount = topicCount;
  697. this.loadRebalancerListener = loadRebalancerListener;
  698. }
  699. public void handleNewSession() throws Exception {
  700. //Called after the zookeeper session has expired and a new session has been created. You would have to re-create
  701. // any ephemeral nodes here.
  702. //
  703. /**
  704. * When we get a SessionExpired event, we lost all ephemeral nodes and zkclient has
  705. * reestablished a connection for us. We need to release the ownership of the
  706. * current consumer and re-register this consumer in the consumer registry and
  707. * trigger a rebalance.
  708. */
  709. logger.info("Zk expired; release old broker partition ownership; re-register consumer " + consumerIdString);
  710. loadRebalancerListener.resetState();
  711. registerConsumerInZK(zkGroupDirs, consumerIdString, topicCount);
  712. //explicitly trigger load balancing for this consumer
  713. loadRebalancerListener.syncedRebalance();
  714. //
  715. // There is no need to resubscribe to child and state changes.
  716. // The child change watchers will be set inside rebalance when we read the children list.
  717. }
  718. public void handleStateChanged(KeeperState state) throws Exception {
  719. }
  720. }
  721. /**
  722. * register consumer data in zookeeper
  723. * <p>
  724. * register path: /consumers/groupid/ids/groupid-consumerid <br/>
  725. * data: {topic:count,topic:count}
  726. * </p>
  727. *
  728. * @param zkGroupDirs zookeeper group path
  729. * @param consumerIdString groupid-consumerid
  730. * @param topicCount topic count
  731. */
  732. private void registerConsumerInZK(ZkGroupDirs zkGroupDirs, String consumerIdString, TopicCount topicCount) {
  733. final String path = zkGroupDirs.consumerRegistryDir + "/" + consumerIdString;
  734. final String data = topicCount.toJsonString();
  735. logger.info(format("register consumer in zookeeper [%s] => [%s]", path, data));
  736. ZkUtils.createEphemeralPathExpectConflict(zkClient, path, data);
  737. }
  738. }