From 2bfef81476d6a2cdee24d8e8b0859c6b8f141cac Mon Sep 17 00:00:00 2001 From: majun87 Date: Sat, 12 Dec 2020 11:55:07 +0800 Subject: [PATCH 01/15] https://github.com/chubaostream/joyqueue/issues/322 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit archive issue: consume log exception #322 归档功能异常:消费归档功能里,在批量进行读取消费记录日志文件的时候,如果读取是在两个文件里进行的(比如批量1000条,500条读取完后再进行下一个文件的500条读取),操作hbase存储如果出现异常的话进行位置回退处理,此时的回退处理是对1000条整体位置的偏移量进行回退,但偏移量横跨两个文件进行读取计算的,所以会导致回退偏移量设置异常,从而影响整个消费记录日志的存储线程。 --- .../broker/archive/ArchiveConfig.java | 18 + .../broker/archive/ArchiveConfigKey.java | 6 +- .../archive/ArchiveRateLimiterManager.java | 106 ++++++ .../broker/archive/ConsumeArchiveService.java | 194 ++++++---- .../broker/archive/ProduceArchiveService.java | 46 ++- .../broker/consumer/ConsumeConfig.java | 10 + .../broker/consumer/ConsumeConfigKey.java | 2 + .../broker/consumer/PartitionConsumption.java | 12 +- .../broker/limit/SubscribeRateLimiter.java | 18 + .../AbstractSubscribeRateLimiterManager.java | 141 +++++++ .../support/DefaultArchiveMonitorService.java | 2 +- .../broker/producer/ProduceConfig.java | 10 + .../broker/producer/ProducerConfigKey.java | 2 + .../broker/retry/BrokerRetryManager.java | 6 +- .../retry/BrokerRetryRateLimiterManager.java | 135 ++----- .../broker/retry/RetryRateLimiter.java | 18 - .../archive/ConsumeArchiveServiceTest.java | 350 +++++++++++++++++- .../archive/store/hbase/MockArchiveStore.java | 80 ++++ 18 files changed, 926 insertions(+), 230 deletions(-) create mode 100644 joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java create mode 100644 joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/SubscribeRateLimiter.java create mode 100644 joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java delete mode 100644 joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/RetryRateLimiter.java create mode 100644 joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/store/hbase/MockArchiveStore.java diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfig.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfig.java index 3e6854900..ca12beaa6 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfig.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfig.java @@ -25,6 +25,9 @@ * Created by chengzhiliang on 2018/12/6. */ public class ArchiveConfig { + public static final String LOG_DETAIL_PRODUCE_PREFIX = "produce."; + public static final String LOG_DETAIL_CONSUME_PREFIX = "consume."; + private static final String ARCHIVE_PATH ="/archive/"; private PropertySupplier propertySupplier; private String archivePath; @@ -60,6 +63,13 @@ public void setPath(String path) { } } + public boolean getLogDetail(String archiveType, String brokerId) { + return (boolean) PropertySupplier.getValue(propertySupplier, + ArchiveConfigKey.ARCHIVE_TRACE_LOG.getName() + archiveType + brokerId, + ArchiveConfigKey.ARCHIVE_TRACE_LOG.getType(), + ArchiveConfigKey.ARCHIVE_TRACE_LOG.getValue()); + } + public int getConsumeBatchNum() { return PropertySupplier.getValue(propertySupplier, ArchiveConfigKey.CONSUME_BATCH_NUM); } @@ -91,6 +101,10 @@ public String getNamespace() { return PropertySupplier.getValue(propertySupplier, ArchiveConfigKey.ARCHIVE_STORE_NAMESPACE); } + public int getStoreFialedRetryCount() { + return PropertySupplier.getValue(propertySupplier, ArchiveConfigKey.ARCHIVE_STORE_RETRY_COUNT); + } + public String getTracerType() { return PropertySupplier.getValue(propertySupplier, BrokerConfigKey.TRACER_TYPE); } @@ -102,4 +116,8 @@ public boolean isReamingEnable() { public boolean isBacklogEnable() { return PropertySupplier.getValue(propertySupplier, ArchiveConfigKey.ARCHIVE_BACKLOG_ENABLE); } + + public int getLogRetainDuration() { + return PropertySupplier.getValue(propertySupplier, ArchiveConfigKey.ARCHIVE_LOG_RETAIN_DURATION); + } } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java index 370a43811..095a84cc4 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java @@ -31,9 +31,11 @@ public enum ArchiveConfigKey implements PropertyDef { ARCHIVE_SWITCH("archive.switch", false, Type.BOOLEAN), ARCHIVE_THREAD_POOL_QUEUE_SIZE("archive.thread.pool.queue.size", 10, Type.INT), ARCHIVE_STORE_NAMESPACE("archive.store.namespace", "joyqueue", Type.STRING), - ARCHIVE_REAMING_ENABLE("archive.reaming.enable", false, Type.BOOLEAN), + ARCHIVE_STORE_RETRY_COUNT("archive.store.retry.count", 3, Type.INT), + ARCHIVE_REAMING_ENABLE("archive.reaming.enable", true, Type.BOOLEAN), ARCHIVE_BACKLOG_ENABLE("archive.backlog.enable", false, Type.BOOLEAN), - + ARCHIVE_TRACE_LOG("archive.trace.log.", false, Type.BOOLEAN), + ARCHIVE_LOG_RETAIN_DURATION("archive.log.retain.duration", 24, Type.INT) ; private String name; diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java new file mode 100644 index 000000000..91537da70 --- /dev/null +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java @@ -0,0 +1,106 @@ +package org.joyqueue.broker.archive; + +import org.apache.commons.lang3.StringUtils; +import org.joyqueue.broker.BrokerContext; +import org.joyqueue.broker.consumer.ConsumeConfig; +import org.joyqueue.broker.consumer.ConsumeConfigKey; +import org.joyqueue.broker.limit.RateLimiter; +import org.joyqueue.broker.limit.support.AbstractSubscribeRateLimiterManager; +import org.joyqueue.broker.producer.ProduceConfig; +import org.joyqueue.broker.producer.ProducerConfigKey; +import org.joyqueue.domain.Config; +import org.joyqueue.domain.Subscription; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; + +/** + * @author majun8 + */ +public class ArchiveRateLimiterManager extends AbstractSubscribeRateLimiterManager { + protected static final Logger LOG = LoggerFactory.getLogger(ArchiveRateLimiterManager.class); + + private ProduceConfig produceConfig; + private ConsumeConfig consumeConfig; + + public ArchiveRateLimiterManager(BrokerContext context) { + super(context); + this.produceConfig = new ProduceConfig(context != null ? context.getPropertySupplier() : null);; + this.consumeConfig = new ConsumeConfig(context != null ? context.getPropertySupplier() : null);; + } + + @Override + public int producerLimitRate(String topic, String app) { + int archiveRate = produceConfig.getArchiveRate(topic, app); + if(archiveRate <= 0) { + // get broker level retry rate + archiveRate = produceConfig.getArchiveRate(); + } + return archiveRate; + } + + @Override + public int consumerLimitRate(String topic, String app) { + int archiveRate = consumeConfig.getArchiveRate(topic, app); + if(archiveRate <= 0) { + // get broker level retry rate + archiveRate = consumeConfig.getArchiveRate(); + } + return archiveRate; + } + + @Override + public void cleanRateLimiter(Config config) { + String configKey = config.getKey(); + if (StringUtils.isBlank(configKey)) { + return; + } + + if (StringUtils.equals(configKey, ProducerConfigKey.PRODUCE_ARCHIVE_RATE.getName())) { + for (Map.Entry> topic : subscribeRateLimiters.entrySet()) { + Iterator> subLimiters = topic.getValue().entrySet().iterator(); + while (subLimiters.hasNext()) { + Map.Entry subLimiter = subLimiters.next(); + String subscribe = subLimiter.getKey(); + if (StringUtils.contains(subscribe, Subscription.Type.PRODUCTION.name() + SPLIT)) { + subLimiters.remove(); + } + } + } + } else if (StringUtils.startsWith(configKey, ProducerConfigKey.PRODUCE_ARCHIVE_RATE_PREFIX.getName())) { + String[] keys = StringUtils.split(configKey, "\\."); + if (keys != null && keys.length == 4) { + String topic = keys[2]; + String app = keys[3]; + if (topic != null && app != null) { + cleanRateLimiter(topic, app, Subscription.Type.PRODUCTION); + } + } + } + + if (StringUtils.equals(configKey, ConsumeConfigKey.CONSUME_ARCHIVE_RATE.getName())) { + for (Map.Entry> topic : subscribeRateLimiters.entrySet()) { + Iterator> subLimiters = topic.getValue().entrySet().iterator(); + while (subLimiters.hasNext()) { + Map.Entry subLimiter = subLimiters.next(); + String subscribe = subLimiter.getKey(); + if (StringUtils.contains(subscribe, Subscription.Type.CONSUMPTION.name() + SPLIT)) { + subLimiters.remove(); + } + } + } + } else if (StringUtils.startsWith(configKey, ConsumeConfigKey.CONSUME_ARCHIVE_RATE_PREFIX.getName())) { + String[] keys = StringUtils.split(configKey, "\\."); + if (keys != null && keys.length == 4) { + String topic = keys[2]; + String app = keys[3]; + if (topic != null && app != null) { + cleanRateLimiter(topic, app, Subscription.Type.CONSUMPTION); + } + } + } + } +} diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java index fd34895ee..90e4d7618 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java @@ -42,11 +42,7 @@ import java.nio.MappedByteBuffer; import java.nio.channels.FileChannel; import java.security.GeneralSecurityException; -import java.util.Arrays; -import java.util.Comparator; -import java.util.LinkedList; -import java.util.List; -import java.util.Optional; +import java.util.*; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -101,18 +97,23 @@ protected void validate() throws Exception { this.readConsumeLogThread = LoopThread.builder() .sleepTime(1, 10) .name("ReadAndPutHBase-ConsumeLog-Thread") + .daemon(true) .onException(e -> { - logger.error(e.getMessage(), e); + logger.error("ReadAndPutHBase-readAndWrite error, happened consume log [{}], error position [{}], error length [{}], exception {}, {}", + repository.rFile.getName(), repository.rMap, readByteCounter.get(), e.getMessage(), e); repository.rollBack(readByteCounter.get()); - logger.info("finish rollback."); + logger.info("Consume-archive: finish rollback consume log [{}], rollback position [{}], rollback length [{}].", + repository.rFile.getName(), repository.rMap, readByteCounter.get()); + readByteCounter.set(0); }) .doWork(this::readAndWrite) .build(); this.cleanConsumeLogFileThread = LoopThread.builder() - .sleepTime(1000 * 10, 1000 * 10) + .sleepTime(1000, 1000 * 10) .name("CleanArchiveFile-ConsumeLog-Thread") - .onException(e -> logger.error(e.getMessage(), e)) + .daemon(true) + .onException(e -> logger.error("CleanArchiveFile-cleanAndRollWriteFile error: {}, {}", e.getMessage(), e)) .doWork(this::cleanAndRollWriteFile) .build(); } @@ -124,6 +125,7 @@ protected void doStart() throws Exception { archiveStore.start(); readConsumeLogThread.start(); cleanConsumeLogFileThread.start(); + logger.info("Consume-archive: service started."); } @@ -134,12 +136,13 @@ protected void doStop() { Close.close(cleanConsumeLogFileThread); Close.close(repository); Close.close(archiveStore); + logger.info("Consume-archive: service stopped."); } /** * 读本地文件写归档存储服务 */ - private void readAndWrite() throws JoyQueueException, InterruptedException { + public void readAndWrite() throws JoyQueueException, InterruptedException { // 读信息,一次读指定条数 int readBatchSize; int batchSize=archiveConfig.getConsumeBatchNum(); @@ -149,12 +152,28 @@ private void readAndWrite() throws JoyQueueException, InterruptedException { if (readBatchSize > 0) { long startTime = SystemClock.now(); - // 调用存储接口写数据 - archiveStore.putConsumeLog(list, tracer); + int count = archiveConfig.getStoreFialedRetryCount(); + do { + try { + // 调用存储接口写数据 + archiveStore.putConsumeLog(list, tracer); + break; + } catch (JoyQueueException e) { + logger.error(String.format( + "Consume-archive: store failed for consume logs, exception size: %s, root cause: %s, cause stack: %s", + list.size(), e.getMessage(), e.getCause()), e); + if (--count == 0) { + throw e; + } + Thread.sleep(new Random().nextInt(count) * 1000); + } + } while (count > 0); long endTime = SystemClock.now(); - logger.debug("Write consumeLogs size:{} to archive store, and elapsed time {}ms", list.size(), endTime - startTime); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Consume-archive: write consumeLogs size: {} to archive store, and elapsed time {}ms", list.size(), endTime - startTime); + } int consumeWriteDelay = archiveConfig.getConsumeWriteDelay(); if (consumeWriteDelay > 0) { @@ -162,10 +181,12 @@ private void readAndWrite() throws JoyQueueException, InterruptedException { } } else { - if (repository.rFile != null && repository.rMap != null) { - logger.debug("read file name {}, read position {}", repository.rFile.getName(), repository.rMap.toString()); - } else { - logger.debug("read file is null."); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + if (repository.rFile != null && repository.rMap != null) { + logger.info("Consume-archive: read consume log file {}, read position {}", repository.rFile.getName(), repository.rMap.toString()); + } else { + logger.info("Consume-archive: read consume log file is null."); + } } break; } @@ -175,7 +196,7 @@ private void readAndWrite() throws JoyQueueException, InterruptedException { private void cleanAndRollWriteFile() { // 删除已归档文件 repository.delArchivedFile(); - // 5分钟滚动生成一个新的写文件,旧文件可归档 + // 1天滚动生成一个新的写文件,旧文件可归档 repository.tryFinishCurWriteFile(); } @@ -190,6 +211,9 @@ private List readConsumeLog(int count) { readByteCounter.set(0); List list = new LinkedList<>(); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Consume-archive: begin to read consume log batch: {}", count); + } for (int i = 0; i < count; i++) { byte[] bytes = repository.readOne(); // 读到结尾会返回byte[0] @@ -202,6 +226,9 @@ private List readConsumeLog(int count) { break; } } + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Consume-archive: end to read consume log size: {}", list.size()); + } return list; } @@ -227,11 +254,11 @@ public long getRemainConsumeLogFileNum() { public void appendConsumeLog(Connection connection, MessageLocation[] locations) throws JoyQueueException { if (!isStarted()) { // 没有启动消费归档服务,添加消费日志 - logger.debug("ConsumeArchiveService not be started."); + logger.warn("ConsumeArchiveService not be started."); return; } List logList = convert(connection, locations); - logList.stream().forEach(log -> { + logList.forEach(log -> { // 序列化 ByteBuffer buffer = ArchiveSerializer.write(log); appendLog(buffer); @@ -276,7 +303,7 @@ private byte[] buildMessageId(MessageLocation location) { try { messageIdBytes = Md5.INSTANCE.encrypt(messageId.getBytes(), null); } catch (GeneralSecurityException e) { - logger.error("topic:{}, partition:{}, index:{}, exception:{}", location.getTopic(), location.getPartition(), location.getIndex(), e); + logger.error("Consume-archive: build consume log messageId error, topic:{}, partition:{}, index:{}, exception:{}", location.getTopic(), location.getPartition(), location.getIndex(), e); } return messageIdBytes; } @@ -293,7 +320,7 @@ private synchronized void appendLog(ByteBuffer buffer) { /** * 本地日志日志文件存储 */ - static class ArchiveMappedFileRepository implements Closeable { + class ArchiveMappedFileRepository implements Closeable { // 消费归档文件本地根存储路径 private String baseDir; @@ -309,6 +336,7 @@ static class ArchiveMappedFileRepository implements Closeable { // 读文件 private File rFile; + private File previousCloseReadFile; // 读文件的Mapped private MappedByteBuffer rMap; // 随机读文件 @@ -365,14 +393,14 @@ private void recover() { * @param buffer */ public synchronized void append(ByteBuffer buffer) { - position += buffer.limit(); + //position += buffer.limit(); // 首次创建文件 if (rwMap == null) { newMappedRWFile(); // may notify reader position = 0; append(buffer); - } else if (1 + position >= pageSize) { + } else if ((position + 1 + buffer.limit()) >= pageSize) { // 一个文件结束时(1个字节记录开始记录 + 记录长度) 小于 文件长度 rwMap.put(Byte.MAX_VALUE); rwMap = null; @@ -380,15 +408,15 @@ public synchronized void append(ByteBuffer buffer) { } else { // buffer 为空时直接返回 if (buffer.limit() == 0) { - logger.debug("append buffer limit is zero."); + logger.warn("Consume-archive: append buffer limit is zero."); return; } // 先写一个开始标记 rwMap.put(Byte.MIN_VALUE); - // 记录一个标示位占用长度 - position += 1; // 写入记录内容 rwMap.put(buffer); + // 记录一个标示位占用长度 + position += 1 + buffer.limit(); } } @@ -413,7 +441,7 @@ public void newMappedRWFile() { rwFileChannel = rwRaf.getChannel(); rwMap = rwFileChannel.map(FileChannel.MapMode.READ_WRITE, 0, pageSize); } catch (Exception ex) { - logger.error("create and mapped file error.", ex); + logger.error("Consume-archive: create and mapped file error.", ex); } } @@ -423,7 +451,6 @@ public void newMappedRWFile() { */ private void mappedReadOnlyFile() { try { - closeCurrentReadFile(); rRaf = new RandomAccessFile(rFile, "r"); rFileChannel = rRaf.getChannel(); rMap = rFileChannel.map(FileChannel.MapMode.READ_ONLY, 0, pageSize); @@ -446,21 +473,21 @@ public byte[] readOne() { } } // 检查一条消费日志开始标记 - if (checkStartFlag(rMap)) { + if (checkPositionReadable(rMap)) { int msgLen = rMap.getInt(); byte[] bytes = new byte[msgLen]; rMap.get(bytes); return bytes; } else if (checkFileEndFlag(rMap)) { - logger.debug("Finish reading the file {}.{}", rFile, rMap.toString()); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Consume-archive: finish read consume log file: {}, position: {}", rFile, rMap.toString()); + } - // 换文件 - if (nextFile() != null) { - // 映射新文件 - mappedReadOnlyFile(); - // 继续读取消息 - return readOne(); + try { + closeCurrentReadFile(); + } catch (IOException e) { + logger.error("Consume-archive: close current consume log archive file: {}, error: {}", rFile, e); } } return new byte[0]; @@ -482,6 +509,20 @@ private boolean checkStartFlag(MappedByteBuffer rMap) { return false; } + private boolean checkPositionReadable(MappedByteBuffer rMap) { + if (rwFile == null || !rwFile.exists()) { + return checkStartFlag(rMap); + } + if (rwFile.getName().equals(rFile.getName())) { + if (rMap.position() < position) { + return checkStartFlag(rMap); + } else { + return false; + } + } + return checkStartFlag(rMap); + } + /** * 检查是否到文件结束 * @@ -504,10 +545,16 @@ private boolean checkFileEndFlag(MappedByteBuffer rMap) { * * @param interval */ - private void rollBack(int interval) { + public void rollBack(int interval) { if (rMap != null) { int position = rMap.position(); int newPosition = position - interval; + if (ConsumeArchiveService.this.archiveConfig.getLogDetail( + ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, + ConsumeArchiveService.this.clusterManager.getBrokerId().toString())) { + logger.info("Consume-archive: read consume log rollback, position: [{}], offset: [{}], reset position: [{}]", + position, interval, newPosition); + } rMap.position(newPosition); } } @@ -522,7 +569,7 @@ public void close() { closeCurrentReadFile(); closeCurrentWriteFile(); } catch (IOException e) { - logger.error("delete read file error.", e); + logger.error("Consume-archive: close consume log read&write files error: {}", e); } } @@ -536,6 +583,10 @@ public void closeCurrentReadFile() throws IOException { if (rRaf != null) { rRaf.close(); } + if (rMap != null) { + rMap = null; + } + previousCloseReadFile = rFile; } /** @@ -551,6 +602,9 @@ public void closeCurrentWriteFile() throws IOException { if (rwRaf != null) { rwRaf.close(); } + if (rwMap != null) { + rwMap = null; + } } @@ -562,31 +616,33 @@ public void closeCurrentWriteFile() throws IOException { private File nextFile() { File file = new File(baseDir); String[] list = file.list(); - if (list == null || list.length == 1) { - logger.debug("only one write file."); - // 归档文件目录下的没有文件,或者文件数等于1,则表示没有可归档文件,返回null + if (list == null) { + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Consume-archive: find no consume log files"); + } return null; } - logger.debug("archive file list {}", Arrays.toString(list)); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Consume-archive: find consume log file list: {}", Arrays.toString(list)); + } - final String concurrentFileName = rFile == null ? "" : rFile.getName(); - List sorted = Arrays.asList(list).stream().filter(name -> name.compareTo(concurrentFileName) > 0) + final String previousCloseReadFileName = previousCloseReadFile == null ? "" : previousCloseReadFile.getName(); + List sorted = Arrays.stream(list).filter(name -> name.compareTo(previousCloseReadFileName) > 0) .sorted(Comparator.naturalOrder()).collect(Collectors.toList()); - if (sorted.size() > 1) { - // 未归档文件数大于1,获取第1个未归档文件 + if (sorted.size() > 0) { + // 只要有一个比上一次关闭的文件新(正常情况下新生成的) 或者就只有一个文件(该文件可能因为broker重启后重新打开) String fileName = sorted.get(0); File tempFile = new File(baseDir + fileName); rFile = tempFile; - logger.debug("current read consume event file {}",tempFile); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Consume-archive: find earliest consume log file: {}",tempFile); + } return rFile; - } else { - logger.debug("only one write file."); } return null; - } /** @@ -600,9 +656,12 @@ private File LastFile() { if (list == null) { return null; } - Optional first = Arrays.asList(list).stream().sorted(Comparator.reverseOrder()).findFirst(); - if (first.isPresent()) { - String fileName = first.get(); + Optional last = Arrays.stream(list) + .map(name -> new File(baseDir + name)) + .filter(f -> !f.isDirectory()) + .map(File::getName).max(Comparator.naturalOrder()); + if (last.isPresent()) { + String fileName = last.get(); File tempFile = new File(baseDir + fileName); rwFile = tempFile; return rwFile; @@ -617,9 +676,7 @@ private void delArchivedFile() { // 遍历删除已归档文件 List archivedFileList = getArchivedFileList(); if (CollectionUtils.isNotEmpty(archivedFileList)) { - archivedFileList.stream().forEach(fileName -> { - new File(baseDir + fileName).delete(); - }); + archivedFileList.forEach(fileName -> new File(baseDir + fileName).delete()); } } @@ -629,19 +686,22 @@ private void delArchivedFile() { * @return */ private List getArchivedFileList() { - // 没有调用到readOne方法,不会初始化rFile,防止空指针,加一下判断 - if (rFile == null) { - logger.debug("Can not get archive file list cause by consume archive read file have no init."); - return null; - } - File file = new File(baseDir); String[] list = file.list(); if (list == null) { return null; } + + // 没有调用到readOne方法,不会初始化rFile,防止空指针,加一下判断 + if (rFile == null) { + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Consume-archive: there is no archive consume log file list cause current broker is not open archive flag."); + } + return null; + } + // 返回已完成归档的文件集合 - return Arrays.asList(list).stream().filter(name -> name.compareTo(rFile.getName()) < 0).collect(Collectors.toList()); + return Arrays.stream(list).filter(name -> name.compareTo(rFile.getName()) < 0).collect(Collectors.toList()); } /** @@ -695,13 +755,13 @@ public synchronized void tryFinishCurWriteFile() { // 5分钟滚动生成一个新的写文件 String name = rwFile.getName(); long now = SystemClock.now(); - // position > 0 说明有归档记录写入文件,并且5分钟没有写满 - if (position > 0 && now - Long.parseLong(name) >= 1000 * 60 * 1) { + // position > 0 说明有归档记录写入文件,并且1天没有写满 + if (position > 0 && now - Long.parseLong(name) >= archiveConfig.getLogRetainDuration() * 1000 * 60 * 60) { // 直接将当前写文件的位置设置为文件大小,下次一次append的时候会新建一个文件继续写 position = pageSize; append(ByteBuffer.wrap(new byte[0])); - logger.info("reset write file {} position {} to pageSize.", rwFile.getName(), rwMap.toString()); + logger.info("Consume-archive: reset write file {} position {} to pageSize.", rwFile.getName(), rwMap.toString()); } } } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java index b598a1102..3015848d8 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java @@ -134,7 +134,8 @@ protected void validate() throws Exception { this.updateItemThread = LoopThread.builder() .sleepTime(1000 * 10, 1000 * 10) .name("UpdateArchiveItem-Thread") - .onException(e -> logger.warn("Exception:", e)) + .daemon(true) + .onException(e -> logger.error("UpdateArchiveItem-Thread error: {}", e)) .doWork(() -> { // 更新item列表 updateArchiveItem(); @@ -145,7 +146,8 @@ protected void validate() throws Exception { this.readMsgThread = LoopThread.builder() .sleepTime(0, 10) .name("ReadArchiveMsg-Thread") - .onException(e -> logger.warn("Exception:", e)) + .daemon(true) + .onException(e -> logger.error("ReadArchiveMsg-Thread error: {}", e)) .doWork(() -> { // 消费接口读取消息,放入队列 readArchiveMsg(); @@ -154,7 +156,8 @@ protected void validate() throws Exception { this.writeMsgThread = LoopThread.builder() .sleepTime(10, 10) .name("WriteArchiveMsg-Thread") - .onException(e -> logger.warn("Exception:", e)) + .daemon(true) + .onException(e -> logger.error("WriteArchiveMsg-Thread error: {}", e)) .doWork(() -> { // 队列读取消息,放入归档存储 write2Store(); @@ -172,7 +175,7 @@ protected void doStart() throws Exception { updateItemThread.start(); readMsgThread.start(); writeMsgThread.start(); - logger.info("produce archive archiveService started."); + logger.info("Produce-archive: service started."); } @Override @@ -183,7 +186,7 @@ protected void doStop() { Close.close(writeMsgThread); Close.close(executorService); Close.close(archiveStore); - logger.info("produce archive archiveService stopped."); + logger.info("Produce-archive: service stopped."); } /** @@ -196,7 +199,9 @@ private void updateArchiveItem() throws JoyQueueException { TopicName name = topicConfig.getName(); // 检查是否开启发送归档 if (clusterManager.checkArchiveable(name)) { - logger.info("Topic:{} send archive is enable.", name.getFullName()); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_PRODUCE_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Produce-archive: topic [{}] archive is enable.", name.getFullName()); + } List partitionSet = clusterManager.getLocalPartitions(topicConfig); partitionSet.stream().forEach(partition -> { list.add(new SendArchiveItem(name.getFullName(), partition)); @@ -207,7 +212,7 @@ private void updateArchiveItem() throws JoyQueueException { itemList.addAndUpdate(list); long count=updateArchiveMetadataCounter.getAndIncrement(); if(count% ARCHIVE_METADATA_MOD ==0){ - logger.info("Add or Update archive item ping,size {}",list.size()); + logger.info("Produce-archive: add or update archive item,size {}",list.size()); } } @@ -226,10 +231,8 @@ private void readArchiveMsg() throws Exception { try { pullResult = consume.getMessage(item.topic, item.partition, readIndex, batchNum); } catch (Throwable th) { - if (logger.isDebugEnabled()) { - logger.debug("read message from topic:" + item.topic + " partition:" + item.partition - + " index:" + item.getReadIndex() + " error.", th); - } + logger.error("Produce-archive: read journal message from topic: [{}] partition: [{}] index: [{}] error: {}, {}", + item.topic, item.partition, item.getReadIndex(), th.getMessage(), th.getCause()); if (th.getCause() instanceof PositionUnderflowException) { // 如果读取位置小于存储索引的最小位置,将位置重置为可读到的最小位置 @@ -237,7 +240,7 @@ private void readArchiveMsg() throws Exception { long minIndex = consume.getMinIndex(new Consumer(item.topic, ""), item.partition); item.setReadIndex(minIndex); - logger.debug("repair read message position SendArchiveItem info:[{}], currentIndex:[{}]", item, minIndex); + logger.error("Produce-archive: repair read journal message position SendArchiveItem info:[{}], currentIndex-min:[{}], exception: {}", item, minIndex, th.getCause()); } @@ -247,7 +250,7 @@ private void readArchiveMsg() throws Exception { long maxIndex = consume.getMaxIndex(new Consumer(item.topic, ""), item.partition); item.setReadIndex(maxIndex); - logger.debug("repair read message position SendArchiveItem info:[{}], currentIndex:[{}]", item, maxIndex); + logger.error("Produce-archive: repair read journal message position SendArchiveItem info:[{}], currentIndex-max:[{}], exception: {}", item, maxIndex, th.getCause()); } @@ -269,8 +272,8 @@ private void readArchiveMsg() throws Exception { item.setReadIndex(readIndex + size, readIndex); // 计数 counter += messageSize; - if (logger.isDebugEnabled()) { - logger.debug("produce archive: {} messages put into the archive queue.", size); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_PRODUCE_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Produce-archive: {} messages put into the archive queue.", size); } } } @@ -388,10 +391,13 @@ private void write2Store() throws InterruptedException { try { // 写入存储 archiveStore.putSendLog(sendLogs, tracer); - logger.debug("Write sendLogs size:{} to archive store.", sendLogs.size()); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_PRODUCE_PREFIX, clusterManager.getBrokerId().toString())) { + logger.info("Produce-archive: write sendLogs size:{} to archive store.", sendLogs.size()); + } // 写入计数(用于归档位置) writeCounter(sendLogs); } catch (JoyQueueException e) { + logger.error("Produce-archive: write sendLogs error: {}", e); // 写入存储失败 hasStoreError.set(true); // 回滚读取位置 @@ -663,9 +669,9 @@ public void addAndUpdate(List newItemList) throws JoyQueueExcep if (!newItemList.contains(item)) { try { remove(item); - logger.info("Clean up archive item,topic {},partition {}",item.getTopic(),item.getPartition()); + logger.info("Produce-archive: clean up archive item, topic [{}], partition [{}]",item.getTopic(),item.getPartition()); }catch (JoyQueueException e){ - logger.info("remove archive item exception",e); + logger.error("Produce-archive: clean up archive item error: {}", e); } } }); @@ -680,9 +686,9 @@ public void addAndUpdate(List newItemList) throws JoyQueueExcep // fullName consumer.setTopic(item.getTopic()); index = consume.getMaxIndex(consumer,item.getPartition()); - logger.info("New archive item,topic {},partition {},init from local store max index {}",item.getTopic(),item.getPartition(),item.getReadIndex()); + logger.info("Produce-archive: new archive item, topic [{}], partition [{}], init from local store max index {}",item.getTopic(),item.getPartition(),item.getReadIndex()); }else{ - logger.info("New archive item,topic {},partition {},recover from archive store,index {}",item.getTopic(),item.getPartition(),index); + logger.info("Produce-archive: new archive item, topic [{}], partition [{}], recover from archive position store index {}",item.getTopic(),item.getPartition(),index); } item.setReadIndex(index); cpList.add(item); diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfig.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfig.java index d1999fe51..5fffa7b72 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfig.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfig.java @@ -88,6 +88,16 @@ public boolean getRetryForceAck(String topic, String app) { ConsumeConfigKey.RETRY_FORCE_ACK_PREFIX.getValue()); } + public int getArchiveRate() { + return propertySupplier.getValue(ConsumeConfigKey.CONSUME_ARCHIVE_RATE); + } + + public int getArchiveRate(String topic, String app) { + return PropertySupplier.getValue(propertySupplier,ConsumeConfigKey.CONSUME_ARCHIVE_RATE_PREFIX.getName() + String.format("%s.%s",topic,app), + ConsumeConfigKey.CONSUME_ARCHIVE_RATE_PREFIX.getType(), ConsumeConfigKey.CONSUME_ARCHIVE_RATE_PREFIX.getValue() + ); + } + public void setConsumePositionPath(String consumePositionPath) { this.consumePositionPath = consumePositionPath; } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java index 1dc3c4a69..4c30a63d1 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java @@ -30,6 +30,8 @@ public enum ConsumeConfigKey implements PropertyDef { RETRY_RATE_PREFIX("retry.rate.", -1, Type.INT), RETRY_FORCE_ACK("retry.ack.force", false, Type.BOOLEAN), RETRY_FORCE_ACK_PREFIX("retry.ack.force.", false, Type.BOOLEAN), + CONSUME_ARCHIVE_RATE("archive.rate.consume", -1, Type.INT), + CONSUME_ARCHIVE_RATE_PREFIX("archive.rate.consume.", -1, Type.INT), BROADCAST_INDEX_RESET_ENABLE("consume.broadcast.index.reset.enable", true, Type.BOOLEAN), BROADCAST_INDEX_RESET_INTERVAL("consume.broadcast.index.reset.interval", 1000 * 60 * 5, Type.INT), BROADCAST_INDEX_RESET_TIME("consume.broadcast.index.reset.time", 1000 * 60 * 60 * 24 * 2, Type.INT), diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/PartitionConsumption.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/PartitionConsumption.java index 023f5265e..9e1a80d5b 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/PartitionConsumption.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/PartitionConsumption.java @@ -213,7 +213,7 @@ protected PullResult getMsgByPartitionAndIndex(Consumer consumer, int group, sho PullResult pullResult = new PullResult(consumer, (short) -1, new ArrayList<>(0)); try { PullResult readResult = getMsgByPartitionAndIndex(consumer.getTopic(), group, partition, index, count); - if (readResult.getBuffers() == null) { + if (readResult.getBuffers() == null || readResult.getBuffers().size() == 0) { // 没有拉到消息直接返回 return pullResult; } @@ -249,9 +249,17 @@ protected PullResult getMsgByPartitionAndIndex(Consumer consumer, int group, sho protected PullResult getMsgByPartitionAndIndex(String topic, int group, short partition, long index, int count) throws JoyQueueException, IOException { long startTime = System.nanoTime(); - PullResult result = new PullResult(topic, null, partition, null); + PullResult result = new PullResult(topic, null, partition, new ArrayList<>(0)); PartitionGroupStore store = storeService.getStore(topic, group); + if (index == store.getRightIndex(partition)) { + return result; + } + + /*if (index < store.getLeftIndex(partition) || index >= store.getRightIndex(partition)) { + return result; + }*/ + ReadResult readRst = store.read(partition, index, count, Long.MAX_VALUE); if (readRst.getCode() == JoyQueueCode.SUCCESS) { diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/SubscribeRateLimiter.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/SubscribeRateLimiter.java new file mode 100644 index 000000000..1fc36ff76 --- /dev/null +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/SubscribeRateLimiter.java @@ -0,0 +1,18 @@ +package org.joyqueue.broker.limit; + +import org.joyqueue.domain.Subscription; +import org.joyqueue.event.MetaEvent; +import org.joyqueue.toolkit.concurrent.EventListener; + +/** + * Subscribe limiter interface + * + **/ +public interface SubscribeRateLimiter extends EventListener { + + /** + * Get or create a rate limiter according to subscribe type + * @return null indicate no limit + **/ + RateLimiter getOrCreate(String topic, String app, Subscription.Type subscribe); +} diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java new file mode 100644 index 000000000..3b3a0a515 --- /dev/null +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java @@ -0,0 +1,141 @@ +package org.joyqueue.broker.limit.support; + +import com.google.common.collect.Maps; +import org.joyqueue.broker.BrokerContext; +import org.joyqueue.broker.cluster.ClusterManager; +import org.joyqueue.broker.limit.RateLimiter; +import org.joyqueue.broker.limit.SubscribeRateLimiter; +import org.joyqueue.domain.Config; +import org.joyqueue.domain.Subscription; +import org.joyqueue.event.MetaEvent; +import org.joyqueue.nsr.event.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** + * @author majun8 + */ +public abstract class AbstractSubscribeRateLimiterManager implements SubscribeRateLimiter { + protected static final Logger LOG = LoggerFactory.getLogger(AbstractSubscribeRateLimiterManager.class); + + protected static final String SPLIT = "."; + + protected ClusterManager clusterManager; + protected ConcurrentMap> subscribeRateLimiters = Maps.newConcurrentMap(); + + public AbstractSubscribeRateLimiterManager(BrokerContext context) { + this.clusterManager = context.getClusterManager(); + this.clusterManager.addListener(this); + } + + public RateLimiter getOrCreate(String topic, String app, Subscription.Type subscribe) { + ConcurrentMap topicRateLimiters = subscribeRateLimiters.get(topic); + if (topicRateLimiters == null) { + topicRateLimiters = new ConcurrentHashMap<>(); + ConcurrentMap old = subscribeRateLimiters.putIfAbsent(topic, topicRateLimiters); + if (old != null) { + topicRateLimiters = old; + } + } + RateLimiter subscribeRateLimiter = topicRateLimiters.get(subscribe.name() + SPLIT + app); + if (subscribeRateLimiter == null) { + switch (subscribe) { + case PRODUCTION: + int pTps = producerLimitRate(topic, app); + if (pTps > 0) { + subscribeRateLimiter = new DefaultRateLimiter(pTps); + RateLimiter oldRateLimiter = topicRateLimiters.putIfAbsent(subscribe.name() + SPLIT + app, subscribeRateLimiter); + if (oldRateLimiter != null) { + subscribeRateLimiter = oldRateLimiter; + } else { + LOG.info("New produce archive rate limiter for {},{},{},{}", topic, app, subscribe.name(), pTps); + } + } + break; + case CONSUMPTION: + int cTps = consumerLimitRate(topic, app); + if (cTps > 0) { + subscribeRateLimiter = new DefaultRateLimiter(cTps); + RateLimiter oldRateLimiter = topicRateLimiters.putIfAbsent(subscribe.name() + SPLIT + app, subscribeRateLimiter); + if (oldRateLimiter != null) { + subscribeRateLimiter = oldRateLimiter; + } else { + LOG.info("New consume archive rate limiter for {},{},{},{}", topic, app, subscribe.name(), cTps); + } + } + break; + } + + } + return subscribeRateLimiter; + } + + public abstract int producerLimitRate(String topic, String app); + + public abstract int consumerLimitRate(String topic, String app); + + @Override + public void onEvent(MetaEvent event) { + switch (event.getEventType()) { + case UPDATE_CONFIG: { + UpdateConfigEvent updateConfigEvent = (UpdateConfigEvent) event; + Config config = updateConfigEvent.getNewConfig(); + cleanRateLimiter(config); + break; + } + case REMOVE_CONFIG: { + RemoveConfigEvent removeConfigEvent = (RemoveConfigEvent) event; + Config config = removeConfigEvent.getConfig(); + cleanRateLimiter(config); + break; + } + case REMOVE_TOPIC: + RemoveTopicEvent topicEvent = (RemoveTopicEvent) event; + cleanRateLimiter(topicEvent.getTopic().getName().getFullName(), null, null); + break; + case UPDATE_PRODUCER: + UpdateProducerEvent updateProducerEvent = (UpdateProducerEvent) event; + cleanRateLimiter(updateProducerEvent.getTopic().getFullName(), + updateProducerEvent.getNewProducer().getApp(), + Subscription.Type.PRODUCTION); + break; + case REMOVE_PRODUCER: + RemoveProducerEvent removeProducerEvent = (RemoveProducerEvent) event; + cleanRateLimiter(removeProducerEvent.getTopic().getFullName(), + removeProducerEvent.getProducer().getApp(), + Subscription.Type.PRODUCTION + ); + break; + case UPDATE_CONSUMER: + UpdateConsumerEvent updateConsumerEvent = (UpdateConsumerEvent) event; + cleanRateLimiter(updateConsumerEvent.getTopic().getFullName(), + updateConsumerEvent.getNewConsumer().getApp(), + Subscription.Type.CONSUMPTION); + break; + case REMOVE_CONSUMER: + RemoveConsumerEvent removeConsumerEvent = (RemoveConsumerEvent) event; + cleanRateLimiter(removeConsumerEvent.getTopic().getFullName(), + removeConsumerEvent.getConsumer().getApp(), + Subscription.Type.CONSUMPTION + ); + break; + } + } + + public abstract void cleanRateLimiter(Config config); + + public void cleanRateLimiter(String topic, String app, Subscription.Type subscribe) { + if (app == null) { + subscribeRateLimiters.remove(topic); + } else { + Map rateLimiters = subscribeRateLimiters.get(topic); + if (rateLimiters != null) { + rateLimiters.remove(subscribe.name() + SPLIT + app); + } + } + } +} diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/monitor/service/support/DefaultArchiveMonitorService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/monitor/service/support/DefaultArchiveMonitorService.java index e2b7eb509..497f3a940 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/monitor/service/support/DefaultArchiveMonitorService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/monitor/service/support/DefaultArchiveMonitorService.java @@ -53,7 +53,7 @@ public ArchiveMonitorInfo getArchiveMonitorInfo() { long sendBackLogNum = getSendBackLogNum(); ArchiveMonitorInfo info = new ArchiveMonitorInfo(); info.setConsumeBacklog(consumeBacklogNum); - info.setConsumeBacklog(sendBackLogNum); + info.setProduceBacklog(sendBackLogNum); info.setTopicProduceBacklog(archiveManager.getSendBacklogNumByTopic()); return info; diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProduceConfig.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProduceConfig.java index ee6a0b272..079897fb7 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProduceConfig.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProduceConfig.java @@ -84,4 +84,14 @@ public int getBodyLength() { public boolean getLogDetail(String app) { return brokerConfig.getLogDetail(app); } + + public int getArchiveRate() { + return propertySupplier.getValue(ProducerConfigKey.PRODUCE_ARCHIVE_RATE); + } + + public int getArchiveRate(String topic, String app) { + return PropertySupplier.getValue(propertySupplier,ProducerConfigKey.PRODUCE_ARCHIVE_RATE_PREFIX.getName() + String.format("%s.%s",topic,app), + ProducerConfigKey.PRODUCE_ARCHIVE_RATE_PREFIX.getType(), ProducerConfigKey.PRODUCE_ARCHIVE_RATE_PREFIX.getValue() + ); + } } \ No newline at end of file diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java index ab888256f..362d6abc1 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java @@ -28,6 +28,8 @@ public enum ProducerConfigKey implements PropertyDef { TOPIC_QOS_LEVEL_PREFIX("produce.topic.qos.level.", -1, Type.INT), APP_QOS_LEVEL_PREFIX("produce.app.qos.level.", -1, Type.INT), PRINT_METRIC_INTERVAL_MS("print.metric.interval", 0L ,Type.LONG), + PRODUCE_ARCHIVE_RATE("archive.rate.produce", -1, Type.INT), + PRODUCE_ARCHIVE_RATE_PREFIX("archive.rate.produce.", -1, Type.INT), // businessId长度 PRODUCE_BUSINESSID_LENGTH("produce.businessId.length", 100, PropertyDef.Type.INT), diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java index bdcbfa353..32719cc5e 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java @@ -24,11 +24,13 @@ import org.joyqueue.broker.Plugins; import org.joyqueue.broker.cluster.ClusterManager; import org.joyqueue.broker.limit.RateLimiter; +import org.joyqueue.broker.limit.SubscribeRateLimiter; import org.joyqueue.broker.monitor.BrokerMonitor; import org.joyqueue.broker.network.support.BrokerTransportClientFactory; import org.joyqueue.config.BrokerConfigKey; import org.joyqueue.domain.Broker; import org.joyqueue.domain.Consumer; +import org.joyqueue.domain.Subscription; import org.joyqueue.domain.TopicName; import org.joyqueue.event.EventType; import org.joyqueue.event.MetaEvent; @@ -85,7 +87,7 @@ public class BrokerRetryManager extends Service implements MessageRetry, B // 集群管理 private ClusterManager clusterManager; private PropertySupplier propertySupplier; - private RetryRateLimiter rateLimiterManager; + private SubscribeRateLimiter rateLimiterManager; private BrokerMonitor brokerMonitor; private PointTracer tracer; @@ -231,7 +233,7 @@ public void addRetry(List retryMessageModelList) throws JoyQu **/ public boolean retryTokenAvailable(Set consumers){ for(Joint consumer:consumers) { - RateLimiter rateLimiter= rateLimiterManager.getOrCreate(consumer.getTopic(),consumer.getApp()); + RateLimiter rateLimiter= rateLimiterManager.getOrCreate(consumer.getTopic(),consumer.getApp(), Subscription.Type.CONSUMPTION); if(rateLimiter==null||rateLimiter.tryAcquireTps()){ return true; } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryRateLimiterManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryRateLimiterManager.java index db220cca7..afcba1080 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryRateLimiterManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryRateLimiterManager.java @@ -1,152 +1,65 @@ package org.joyqueue.broker.retry; -import com.google.common.collect.Maps; import org.apache.commons.lang3.StringUtils; import org.joyqueue.broker.BrokerContext; -import org.joyqueue.broker.cluster.ClusterManager; import org.joyqueue.broker.consumer.ConsumeConfig; import org.joyqueue.broker.consumer.ConsumeConfigKey; -import org.joyqueue.broker.limit.RateLimiter; -import org.joyqueue.broker.limit.support.DefaultRateLimiter; +import org.joyqueue.broker.limit.support.AbstractSubscribeRateLimiterManager; import org.joyqueue.domain.Config; -import org.joyqueue.event.MetaEvent; -import org.joyqueue.nsr.event.RemoveConfigEvent; -import org.joyqueue.nsr.event.RemoveConsumerEvent; -import org.joyqueue.nsr.event.RemoveTopicEvent; -import org.joyqueue.nsr.event.UpdateConfigEvent; +import org.joyqueue.domain.Subscription; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - /** * Consumer retry rate limiter manager * **/ -public class BrokerRetryRateLimiterManager implements RetryRateLimiter{ +public class BrokerRetryRateLimiterManager extends AbstractSubscribeRateLimiterManager { protected static final Logger LOG = LoggerFactory.getLogger(BrokerRetryRateLimiterManager.class); - private ClusterManager clusterManager; + private ConsumeConfig consumeConfig; - private ConcurrentMap> retryRateLimiters = Maps.newConcurrentMap(); public BrokerRetryRateLimiterManager(BrokerContext context){ - this.clusterManager = context.getClusterManager(); - this.clusterManager.addListener(this); + super(context); this.consumeConfig=new ConsumeConfig(context != null ? context.getPropertySupplier() : null); } + @Override - public RateLimiter getOrCreate(String topic, String app) { - ConcurrentMap topicRateLimiters=retryRateLimiters.get(topic); - if(topicRateLimiters==null){ - topicRateLimiters=new ConcurrentHashMap(); - ConcurrentMap old=retryRateLimiters.putIfAbsent(topic,topicRateLimiters); - if(old!=null){ - topicRateLimiters=old; - } - } - RateLimiter consumerRetryRateLimiter=topicRateLimiters.get(app); - if(consumerRetryRateLimiter==null){ - int tps=consumerRetryRate(topic,app); - if(tps>0) { // ulimit - consumerRetryRateLimiter = new DefaultRateLimiter(tps); - RateLimiter oldRateLimiter = topicRateLimiters.putIfAbsent(app, consumerRetryRateLimiter); - if (oldRateLimiter != null) { - consumerRetryRateLimiter = oldRateLimiter; - }else{ - LOG.info("New rate limiter for {},{},{}",topic,app,tps); - } - } - } - return consumerRetryRateLimiter; + public int producerLimitRate(String topic, String app) { + return 0; } - - /** - * Mix broker level consume retry rate(default ulimit) with consumer level config - * priority: - * 1. broker consumer level - * 2. broker level - * 3. consumer config level(not support) - * @return -1 indicate ulimit if above all not configure - * - **/ - public int consumerRetryRate(String topic,String app){ - int retryRate=consumeConfig.getRetryRate(topic,app); - if(retryRate<=0){ + @Override + public int consumerLimitRate(String topic, String app) { + int retryRate = consumeConfig.getRetryRate(topic, app); + if (retryRate <= 0) { // get broker level retry rate - retryRate=consumeConfig.getRetryRate(); + retryRate = consumeConfig.getRetryRate(); } return retryRate; } - - /** - * Clean up current rate limiter if consumer metadata has any changes - **/ - @Override - public void onEvent(MetaEvent event) { - switch (event.getEventType()) { - case UPDATE_CONFIG: { - UpdateConfigEvent updateConfigEvent = (UpdateConfigEvent) event; - Config config = updateConfigEvent.getNewConfig(); - cleanRateLimiter(config); - break; - } - case REMOVE_CONFIG: { - RemoveConfigEvent removeConfigEvent = (RemoveConfigEvent) event; - Config config=removeConfigEvent.getConfig(); - cleanRateLimiter(config); - break; - } - case REMOVE_TOPIC: - RemoveTopicEvent topicEvent = (RemoveTopicEvent) event; - cleanRateLimiter(topicEvent.getTopic().getName().getFullName(),null); - break; - case REMOVE_CONSUMER: - RemoveConsumerEvent removeConsumerEvent = (RemoveConsumerEvent) event; - cleanRateLimiter(removeConsumerEvent.getTopic().getFullName(), removeConsumerEvent.getConsumer().getApp()); - break; - } - } - /** * @param config consumer config * **/ - public void cleanRateLimiter(Config config){ - String configKey=config.getKey(); + public void cleanRateLimiter(Config config) { + String configKey = config.getKey(); if (StringUtils.isBlank(configKey)) { return; } - if (configKey.equals(ConsumeConfigKey.RETRY_RATE.getName())) { - retryRateLimiters.clear(); - } else if (configKey.startsWith(ConsumeConfigKey.RETRY_RATE_PREFIX.getName())) { - String[] keys=configKey.split("\\."); - if(keys.length == 4){ - String topic=keys[2]; - String app=keys[3]; - if(topic!=null&&app!=null) { - cleanRateLimiter(topic, app); + if (StringUtils.equals(configKey, ConsumeConfigKey.RETRY_RATE.getName())) { + subscribeRateLimiters.clear(); + } else if (StringUtils.startsWith(configKey, ConsumeConfigKey.RETRY_RATE_PREFIX.getName())) { + String[] keys = StringUtils.split(configKey, "\\."); + if (keys.length == 4) { + String topic = keys[2]; + String app = keys[3]; + if (topic != null && app != null) { + cleanRateLimiter(topic, app, Subscription.Type.CONSUMPTION); } } } } - - /** - * Clean rate limiter of consumer - **/ - public void cleanRateLimiter(String topic,String app) { - if (app == null) { - retryRateLimiters.remove(topic); - }else { - Map rateLimiters = retryRateLimiters.get(topic); - if (rateLimiters != null) { - rateLimiters.remove(app); - } - - } - } } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/RetryRateLimiter.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/RetryRateLimiter.java deleted file mode 100644 index 8b96e144e..000000000 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/RetryRateLimiter.java +++ /dev/null @@ -1,18 +0,0 @@ -package org.joyqueue.broker.retry; - -import org.joyqueue.broker.limit.RateLimiter; -import org.joyqueue.event.MetaEvent; -import org.joyqueue.toolkit.concurrent.EventListener; - -/** - * Retry rate limiter interface - * - **/ -public interface RetryRateLimiter extends EventListener { - - /** - * Get or create a rate limiter - * @return null indicate no limit - **/ - RateLimiter getOrCreate(String topic, String app); -} diff --git a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java index d869dc5e8..97a90e929 100644 --- a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java +++ b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java @@ -15,29 +15,65 @@ */ package org.joyqueue.broker.archive; -import org.joyqueue.broker.archive.ArchiveSerializer; -import org.joyqueue.broker.archive.ConsumeArchiveService; +import org.joyqueue.broker.BrokerContext; +import org.joyqueue.broker.archive.store.hbase.MockArchiveStore; +import org.joyqueue.broker.cluster.ClusterManager; +import org.joyqueue.broker.config.Configuration; +import org.joyqueue.domain.Broker; +import org.joyqueue.exception.JoyQueueException; +import org.joyqueue.server.archive.store.api.ArchiveStore; import org.joyqueue.server.archive.store.model.ConsumeLog; +import org.joyqueue.toolkit.concurrent.LoopThread; +import org.joyqueue.toolkit.config.Property; +import org.joyqueue.toolkit.config.PropertySupplier; import org.joyqueue.toolkit.time.SystemClock; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.lang.reflect.Field; import java.nio.ByteBuffer; -import java.util.Arrays; +import java.nio.MappedByteBuffer; +import java.nio.channels.FileChannel; +import java.util.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; /** * Created by chengzhiliang on 2018/12/19. */ public class ConsumeArchiveServiceTest { - final int writeRecordNum = 1000000; + final int writeRecordNum = 900000; + private AtomicInteger readByteCounter = new AtomicInteger(0); + private ClusterManager clusterManager; + private ArchiveConfig archiveConfig; + private ConsumeArchiveService consumeService; + + @Before + public void init() throws NoSuchFieldException, IllegalAccessException { + PropertySupplier propertySupplier = new Configuration(); + ((Configuration) propertySupplier).addProperty(Property.APPLICATION_DATA_PATH, "/Users/majun8"); + BrokerContext brokerContext = new BrokerContext().propertySupplier(propertySupplier); + clusterManager = new ClusterManager(null, null, null, brokerContext); + Broker broker = new Broker(); + broker.setId(123); + Field brokerField = clusterManager.getClass().getDeclaredField("broker"); + brokerField.setAccessible(true); + brokerField.set(clusterManager, broker); + + archiveConfig = new ArchiveConfig(propertySupplier); + consumeService = new ConsumeArchiveService(archiveConfig, clusterManager); + } @Test public void writeConsumeLog() throws InterruptedException { String testPath = getTestPath(); delTestFolder(testPath); - ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = new ConsumeArchiveService.ArchiveMappedFileRepository(testPath); + ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = consumeService.new ArchiveMappedFileRepository(testPath); String[] appArr = {"test", "app_test", "app_app_test"}; for (int i = 0; i < writeRecordNum; i++) { ConsumeLog consumeLog = new ConsumeLog(); @@ -59,11 +95,32 @@ public void writeConsumeLog() throws InterruptedException { Thread.sleep(100); } + @Test + public void multiWriteConsumeLogTest() { + String testPath = getTestPath(); + ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = consumeService.new ArchiveMappedFileRepository(testPath); + String[] appArr = {"test", "app_test", "app_app_test"}; + for (int i = 0; i < 10; i++) { + new Thread(() -> { + for (int i1 = 0; i1 < writeRecordNum; i1++) { + ConsumeLog consumeLog = new ConsumeLog(); + consumeLog.setBrokerId(i1); + consumeLog.setBytesMessageId(new byte[16]); + consumeLog.setClientIp(new byte[16]); + consumeLog.setConsumeTime(System.currentTimeMillis()); + consumeLog.setApp(appArr[i1 % 3]); + ByteBuffer buffer = ArchiveSerializer.write(consumeLog); + archiveMappedFileRepository.append(buffer); + } + }, "write-thread-" + i).start(); + } + } + @Test public void writeConsumeLog2() throws InterruptedException { String testPath = getTestPath(); delTestFolder(testPath); - ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = new ConsumeArchiveService.ArchiveMappedFileRepository(testPath); + ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = consumeService.new ArchiveMappedFileRepository(testPath); String[] appArr = {"test", "app_test", "app_app_test"}; for (int i = 0; i < 100; i++) { ConsumeLog consumeLog = new ConsumeLog(); @@ -109,7 +166,7 @@ public void writeConsumeLog2() throws InterruptedException { public void readConsumeLog() throws InterruptedException { writeConsumeLog(); String testPath = getTestPath(); - ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = new ConsumeArchiveService.ArchiveMappedFileRepository(testPath); + ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = consumeService.new ArchiveMappedFileRepository(testPath); for (int i = 0; i < writeRecordNum; i++) { byte[] bytes = archiveMappedFileRepository.readOne(); @@ -129,6 +186,7 @@ private void delTestFolder(String testPath) { File file = new File(testPath); if (file.isDirectory()) { File[] files = file.listFiles(); + assert files != null; if (files.length > 0) { Arrays.stream(files).forEach(file1 -> file1.delete()); } @@ -149,4 +207,282 @@ private String getTestPath() { return userPath + childPath; } + private String getTestFile() { + return ""; + } + + @Test + public void testConsumeArchiveService() throws NoSuchFieldException, IllegalAccessException { + ArchiveStore archiveStore = new MockArchiveStore(); + Field archiveField = consumeService.getClass().getDeclaredField("archiveStore"); + archiveField.setAccessible(true); + archiveField.set(consumeService, archiveStore); + + ConsumeArchiveService.ArchiveMappedFileRepository repository = consumeService.new ArchiveMappedFileRepository(archiveConfig.getArchivePath()); + Field repositoryField = consumeService.getClass().getDeclaredField("repository"); + repositoryField.setAccessible(true); + repositoryField.set(consumeService, repository); + + + Field counterField = consumeService.getClass().getDeclaredField("readByteCounter"); + counterField.setAccessible(true); + counterField.set(consumeService, readByteCounter); + + LoopThread readConsumeLogThread = LoopThread.builder() + .sleepTime(1, 10) + .name("ReadAndPutHBase-ConsumeLog-Thread") + .onException(e -> { + System.out.println(e.getMessage()); + repository.rollBack(readByteCounter.get()); + System.out.println("finish rollback."); + }) + .doWork(this::wrapReadAndWrite) + .build(); + Field readThread = consumeService.getClass().getDeclaredField("readConsumeLogThread"); + readThread.setAccessible(true); + readThread.set(consumeService, readConsumeLogThread); + + readConsumeLogThread.start(); + } + + private void wrapReadAndWrite() throws InterruptedException, JoyQueueException { + consumeService.readAndWrite(); + } + + @Test + public void testArchive() { + ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = + consumeService.new ArchiveMappedFileRepository(getTestPath()); + + try { + readBatch(archiveMappedFileRepository); + } catch (Exception e) { + archiveMappedFileRepository.rollBack(readByteCounter.get()); + try { + readBatch(archiveMappedFileRepository); + } catch (Exception e1) { + e1.printStackTrace(); + } + } + System.out.println("Finish"); + } + + private void readBatch(ConsumeArchiveService.ArchiveMappedFileRepository repository) throws Exception { + int readBatchSize; + int batchSize = 1000; + int exception = 0; + do { + List list = readConsumeLog(repository, batchSize); + readBatchSize = list.size(); + if (readBatchSize > 0) { + long startTime = SystemClock.now(); + + // 调用存储接口写数据 + System.out.println("Store batch: " + batchSize); + /*if (++exception == 3) { + throw new Exception("Exception for store"); + }*/ + + long endTime = SystemClock.now(); + + } else { + break; + } + } while (readBatchSize == batchSize); + } + + private List readConsumeLog(ConsumeArchiveService.ArchiveMappedFileRepository repository, int count) { + // 每次读取之前清零 + readByteCounter.set(0); + + List list = new LinkedList<>(); + for (int i = 0; i < count; i++) { + byte[] bytes = repository.readOne(); + // 读到结尾会返回byte[0] + if (bytes.length > 0) { + // 1个字节开始符号,4个字节int类型长度信息 + readByteCounter.addAndGet(1 + 4 + bytes.length); + // 反序列花并放入集合 + list.add(ArchiveSerializer.read(ByteBuffer.wrap(bytes))); + } else { + break; + } + } + return list; + } + + @Test + public void testFilelist() { + File file = new File(archiveConfig.getArchivePath()); + String[] list = file.list(); + if (list != null && list.length > 0) { + System.out.println(Arrays.stream(list).filter(name -> name.compareTo("") > 0).collect(Collectors.toList())); + } + } + + @Test + public void testAsyncWrite() throws IOException { + String path = getTestPath(); + String fileName = getTestFile(); + File wFile = new File(path + fileName); + MappedByteBuffer wMap = new RandomAccessFile(wFile, "rw").getChannel().map(FileChannel.MapMode.READ_WRITE,0, 1024); + wMap.position(100); + wMap.putInt(10); + print(wMap); + } + + @Test + public void testAsyncRead() throws IOException { + String path = getTestPath(); + String fileName = getTestFile(); + File rFile = new File(path + fileName); + MappedByteBuffer rMap = new RandomAccessFile(rFile, "r").getChannel().map(FileChannel.MapMode.READ_ONLY, 0, 1024); + rMap.position(100); + int i100 = rMap.getInt(); + System.out.println(i100); + print(rMap); + int iUnknown = rMap.getInt(); + System.out.println(iUnknown); + print(rMap); + byte[] bytes = new byte[iUnknown]; + rMap.get(bytes); + System.out.println(bytes); + print(rMap); + } + + @Test + public void testMappedFileRollback() throws IOException { + String path = getTestPath(); + String fileName = getTestFile(); + File file = new File(path + fileName); + MappedByteBuffer map = new RandomAccessFile(file, "r").getChannel().map(FileChannel.MapMode.READ_ONLY, 0, 1024 * 1024 * 16);; + print(map); + map.position(6815718); + boolean find = false; + do { + find = rollback(map); + + if (find) { + print(map); + int msgLen = map.getInt(); + byte[] bytes = new byte[msgLen]; + map.get(bytes); + ConsumeLog log = ArchiveSerializer.read(ByteBuffer.wrap(bytes)); + System.out.println(log); + } + } while (!find); + } + + private boolean rollback(MappedByteBuffer map) { + if (map.get() != Byte.MIN_VALUE) { + map.position(map.position() - 2); + return false; + } else { + return true; + } + } + + @Test + public void testMappedFileRW() throws IOException { + String pathR = getTestPath(); + String fileNameR = getTestFile(); + File fileR = new File(pathR + fileNameR); + MappedByteBuffer mapR = new RandomAccessFile(fileR, "r").getChannel().map(FileChannel.MapMode.READ_ONLY, 0, 1024 * 1024 * 16);; + print(mapR); + String pathW = getTestPath(); + String fileNameW = getTestFile(); + File fileW = new File(pathW + fileNameW); + MappedByteBuffer mapW = new RandomAccessFile(fileW, "r").getChannel().map(FileChannel.MapMode.READ_ONLY, 0, 1024 * 1024 * 16);; + print(mapW); + + boolean isExit = true; + do { + if (checkPositionReadable(fileW, fileR, mapR, 5181218)) { + int msgLen = mapR.getInt(); + byte[] bytes = new byte[msgLen]; + mapR.get(bytes); + print(mapR); + } else if (checkFileEndFlag(mapR)) { + isExit = false; + System.out.println("File end, lenght: " + mapR); + break; + } + } while (isExit); + } + + @Test + public void testMappedFileRead() throws IOException { + AtomicInteger counter = new AtomicInteger(0); + String path = getTestPath(); + String fileName = getTestFile(); + File file = new File(path + fileName); + MappedByteBuffer map = new RandomAccessFile(file, "r").getChannel().map(FileChannel.MapMode.READ_ONLY, 0, 1024 * 1024 * 16);; + print(map); + //map.position(5705701); + map.position(2768394); + print(map); + boolean isExit = true; + int exception = 3; + int error = 0; + do { + counter.set(0); + for (int i = 0; i < 1000; i++) { + if (checkStartFlag(map)) { + int msgLen = map.getInt(); + byte[] bytes = new byte[msgLen]; + map.get(bytes); + //print(map); + counter.addAndGet(1 + 4 + bytes.length); + } else if (checkFileEndFlag(map)) { + isExit = false; + System.out.println("File end, lenght: " + counter.get()); + break; + } + } + if (++error % exception == 0) { + print(map); + map.position(map.position() - counter.get()); + } + } while (isExit); + } + + private boolean checkStartFlag(MappedByteBuffer rMap) { + if (rMap.position() + 1 < 1024 * 1024 * 16) { + if (rMap.get() == Byte.MIN_VALUE) { + return true; + } else { + rMap.position(rMap.position() - 1); + } + } + return false; + } + + private boolean checkFileEndFlag(MappedByteBuffer rMap) { + if (rMap.position() + 1 <= 1024 * 1024 * 16) { + if (rMap.get() == Byte.MAX_VALUE) { + rMap.position(rMap.position() - 1); + return true; + } else { + rMap.position(rMap.position() - 1); + } + } + return false; + } + + private boolean checkPositionReadable(File rwFile, File rFile, MappedByteBuffer rMap, int position) { + if (rwFile.getName().equals(rFile.getName())) { + if (rMap.position() <= position) { + return checkStartFlag(rMap); + } else { + return false; + } + } + return checkStartFlag(rMap); + } + + private static void print(ByteBuffer buffer) { + System.out.printf("position: %d, limit: %d, capacity: %d\n", + buffer.position(), buffer.limit(), buffer.capacity()); + } + } \ No newline at end of file diff --git a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/store/hbase/MockArchiveStore.java b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/store/hbase/MockArchiveStore.java new file mode 100644 index 000000000..4d5564e4a --- /dev/null +++ b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/store/hbase/MockArchiveStore.java @@ -0,0 +1,80 @@ +package org.joyqueue.broker.archive.store.hbase; + +import org.joyqueue.exception.JoyQueueCode; +import org.joyqueue.exception.JoyQueueException; +import org.joyqueue.monitor.PointTracer; +import org.joyqueue.server.archive.store.api.ArchiveStore; +import org.joyqueue.server.archive.store.model.AchivePosition; +import org.joyqueue.server.archive.store.model.ConsumeLog; +import org.joyqueue.server.archive.store.model.Query; +import org.joyqueue.server.archive.store.model.SendLog; + +import java.util.List; + +public class MockArchiveStore implements ArchiveStore { + + int exception = 0; + int exceptionCount = 3; + + @Override + public void putConsumeLog(List consumeLogs, PointTracer tracer) throws JoyQueueException { + if (++exception % 3 == 0) { + throw new JoyQueueException("Exception for store", JoyQueueCode.SE_IO_ERROR.getCode()); + } + } + + @Override + public void putSendLog(List sendLogs, PointTracer tracer) throws JoyQueueException { + + } + + @Override + public void putPosition(AchivePosition achivePosition) throws JoyQueueException { + + } + + @Override + public Long getPosition(String topic, short partition) throws JoyQueueException { + return null; + } + + @Override + public void cleanPosition(String topic, short partition) throws JoyQueueException { + + } + + @Override + public List scanSendLog(Query query) throws JoyQueueException { + return null; + } + + @Override + public SendLog getOneSendLog(Query query) throws JoyQueueException { + return null; + } + + @Override + public List scanConsumeLog(String messageId, Integer count) throws JoyQueueException { + return null; + } + + @Override + public void setNameSpace(String nameSpace) { + + } + + @Override + public void start() throws Exception { + + } + + @Override + public void stop() { + + } + + @Override + public boolean isStarted() { + return false; + } +} From 08fe7d5c798b28d473c7ecd81cbfd13d6efdb16b Mon Sep 17 00:00:00 2001 From: majun87 Date: Sun, 13 Dec 2020 15:35:18 +0800 Subject: [PATCH 02/15] =?UTF-8?q?=E6=B3=A8=E9=87=8A=E5=A4=9A=E4=BD=99?= =?UTF-8?q?=E6=B5=8B=E8=AF=95=E7=94=A8=E4=BE=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../archive/ConsumeArchiveServiceTest.java | 35 ++++--------------- 1 file changed, 7 insertions(+), 28 deletions(-) diff --git a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java index 97a90e929..8c5698fb0 100644 --- a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java +++ b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java @@ -40,7 +40,6 @@ import java.nio.channels.FileChannel; import java.util.*; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; /** * Created by chengzhiliang on 2018/12/19. @@ -95,7 +94,7 @@ public void writeConsumeLog() throws InterruptedException { Thread.sleep(100); } - @Test + //@Test public void multiWriteConsumeLogTest() { String testPath = getTestPath(); ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = consumeService.new ArchiveMappedFileRepository(testPath); @@ -211,7 +210,7 @@ private String getTestFile() { return ""; } - @Test + //@Test public void testConsumeArchiveService() throws NoSuchFieldException, IllegalAccessException { ArchiveStore archiveStore = new MockArchiveStore(); Field archiveField = consumeService.getClass().getDeclaredField("archiveStore"); @@ -249,7 +248,7 @@ private void wrapReadAndWrite() throws InterruptedException, JoyQueueException { consumeService.readAndWrite(); } - @Test + //@Test public void testArchive() { ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = consumeService.new ArchiveMappedFileRepository(getTestPath()); @@ -311,27 +310,7 @@ private List readConsumeLog(ConsumeArchiveService.ArchiveMappedFileR return list; } - @Test - public void testFilelist() { - File file = new File(archiveConfig.getArchivePath()); - String[] list = file.list(); - if (list != null && list.length > 0) { - System.out.println(Arrays.stream(list).filter(name -> name.compareTo("") > 0).collect(Collectors.toList())); - } - } - - @Test - public void testAsyncWrite() throws IOException { - String path = getTestPath(); - String fileName = getTestFile(); - File wFile = new File(path + fileName); - MappedByteBuffer wMap = new RandomAccessFile(wFile, "rw").getChannel().map(FileChannel.MapMode.READ_WRITE,0, 1024); - wMap.position(100); - wMap.putInt(10); - print(wMap); - } - - @Test + //@Test public void testAsyncRead() throws IOException { String path = getTestPath(); String fileName = getTestFile(); @@ -350,7 +329,7 @@ public void testAsyncRead() throws IOException { print(rMap); } - @Test + //@Test public void testMappedFileRollback() throws IOException { String path = getTestPath(); String fileName = getTestFile(); @@ -382,7 +361,7 @@ private boolean rollback(MappedByteBuffer map) { } } - @Test + //@Test public void testMappedFileRW() throws IOException { String pathR = getTestPath(); String fileNameR = getTestFile(); @@ -410,7 +389,7 @@ public void testMappedFileRW() throws IOException { } while (isExit); } - @Test + //@Test public void testMappedFileRead() throws IOException { AtomicInteger counter = new AtomicInteger(0); String path = getTestPath(); From 48033442e23dd22af91fd5ff5bdcb7851baaa2e1 Mon Sep 17 00:00:00 2001 From: majun87 Date: Sun, 13 Dec 2020 16:45:26 +0800 Subject: [PATCH 03/15] =?UTF-8?q?=E6=B3=A8=E9=87=8A=E5=A4=9A=E4=BD=99?= =?UTF-8?q?=E6=B5=8B=E8=AF=95=E7=94=A8=E4=BE=8B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../broker/archive/ConsumeArchiveServiceTest.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java index 8c5698fb0..2fb3f4198 100644 --- a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java +++ b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java @@ -161,17 +161,26 @@ public void writeConsumeLog2() throws InterruptedException { } } - @Test + //@Test public void readConsumeLog() throws InterruptedException { writeConsumeLog(); String testPath = getTestPath(); ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = consumeService.new ArchiveMappedFileRepository(testPath); + int brokerid = 0; for (int i = 0; i < writeRecordNum; i++) { byte[] bytes = archiveMappedFileRepository.readOne(); if (bytes.length > 0) { + brokerid = i; ConsumeLog read = ArchiveSerializer.read(ByteBuffer.wrap(bytes)); - Assert.assertEquals(i, read.getBrokerId()); + try { + Assert.assertEquals(brokerid, read.getBrokerId()); + } catch (Throwable e) { + System.out.println("index: " + i + " brokerid: "+ read.getBrokerId()); + throw e; + } + } else { + brokerid -= 1; } } } From e08d567baa4aa6212122eb3f1ad2df72b5bfb608 Mon Sep 17 00:00:00 2001 From: majun87 Date: Sun, 13 Dec 2020 17:24:16 +0800 Subject: [PATCH 04/15] =?UTF-8?q?=E5=A2=9E=E5=8A=A0=E6=B5=8B=E8=AF=95?= =?UTF-8?q?=E8=A6=86=E7=9B=96=E7=8E=87?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java index 2fb3f4198..a364b1aee 100644 --- a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java +++ b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ConsumeArchiveServiceTest.java @@ -257,7 +257,7 @@ private void wrapReadAndWrite() throws InterruptedException, JoyQueueException { consumeService.readAndWrite(); } - //@Test + @Test public void testArchive() { ConsumeArchiveService.ArchiveMappedFileRepository archiveMappedFileRepository = consumeService.new ArchiveMappedFileRepository(getTestPath()); From 778a84d7949254636e4baf79bd2ca75e9a10cf35 Mon Sep 17 00:00:00 2001 From: majun87 Date: Tue, 15 Dec 2020 17:07:06 +0800 Subject: [PATCH 05/15] =?UTF-8?q?data-service=E5=B1=82=E5=8E=BB=E6=8E=89ar?= =?UTF-8?q?chive-hbase=E7=9A=84=E4=BE=9D=E8=B5=96?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../joyqueue-data-service/pom.xml | 4 +- .../service/impl/ArchiveServiceImpl.java | 2 +- .../command/archive/ArchiveCommand.java | 6 +- .../archive/store/query/QueryCondition.java | 122 ++++++++ .../store/utils/ArchiveSerializer.java | 264 ++++++++++++++++++ 5 files changed, 392 insertions(+), 6 deletions(-) create mode 100644 joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/query/QueryCondition.java create mode 100644 joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/utils/ArchiveSerializer.java diff --git a/joyqueue-console/joyqueue-data/joyqueue-data-service/pom.xml b/joyqueue-console/joyqueue-data/joyqueue-data-service/pom.xml index 18c2b09a8..f2661cafb 100644 --- a/joyqueue-console/joyqueue-data/joyqueue-data-service/pom.xml +++ b/joyqueue-console/joyqueue-data/joyqueue-data-service/pom.xml @@ -124,10 +124,10 @@ joyqueue-archive-api - + org.joyqueue joyqueue-client-all diff --git a/joyqueue-console/joyqueue-data/joyqueue-data-service/src/main/java/org/joyqueue/service/impl/ArchiveServiceImpl.java b/joyqueue-console/joyqueue-data/joyqueue-data-service/src/main/java/org/joyqueue/service/impl/ArchiveServiceImpl.java index 7911d5d26..d2f2b859d 100644 --- a/joyqueue-console/joyqueue-data/joyqueue-data-service/src/main/java/org/joyqueue/service/impl/ArchiveServiceImpl.java +++ b/joyqueue-console/joyqueue-data/joyqueue-data-service/src/main/java/org/joyqueue/service/impl/ArchiveServiceImpl.java @@ -24,10 +24,10 @@ import org.joyqueue.model.domain.User; import org.joyqueue.model.query.QApplication; import org.joyqueue.model.query.QArchive; -import org.joyqueue.server.archive.store.QueryCondition; import org.joyqueue.server.archive.store.api.ArchiveStore; import org.joyqueue.server.archive.store.model.ConsumeLog; import org.joyqueue.server.archive.store.model.SendLog; +import org.joyqueue.server.archive.store.query.QueryCondition; import org.joyqueue.service.ApplicationService; import org.joyqueue.service.ArchiveService; import org.joyqueue.service.TopicService; diff --git a/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/archive/ArchiveCommand.java b/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/archive/ArchiveCommand.java index 05ebb2ca1..9b8194ad7 100644 --- a/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/archive/ArchiveCommand.java +++ b/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/archive/ArchiveCommand.java @@ -21,7 +21,7 @@ import org.joyqueue.exception.ServiceException; import org.joyqueue.handler.error.ErrorCode; import org.joyqueue.message.SourceType; -import org.joyqueue.server.archive.store.HBaseSerializer; +import org.joyqueue.server.archive.store.utils.ArchiveSerializer; import org.joyqueue.server.retry.model.RetryMessageModel; import org.joyqueue.handler.Constants; import org.joyqueue.service.MessagePreviewService; @@ -200,10 +200,10 @@ public BrokerMessage filterBrokerMessage(BrokerMessage brokerMessage, SendLog se } for(BrokerMessage m:msgs){ String msgId=ArchiveUtils.messageId(brokerMessage.getTopic(),m.getPartition(),m.getMsgIndexNo()); - byte[] msgIdMd5Bytes=HBaseSerializer.md5(msgId,null); + byte[] msgIdMd5Bytes= ArchiveSerializer.md5(msgId,null); if(logger.isDebugEnabled()) { logger.debug("current message business id {},message id {},md5 length {},base 64 bytes {},hex {}", m.getBusinessId(), msgId, msgIdMd5Bytes.length, - Base64.getEncoder().encodeToString(msgIdMd5Bytes), HBaseSerializer.byteArrayToHexStr(msgIdMd5Bytes)); + Base64.getEncoder().encodeToString(msgIdMd5Bytes), ArchiveSerializer.byteArrayToHexStr(msgIdMd5Bytes)); } if(Arrays.equals(msgIdMd5Bytes,sendLog.getBytesMessageId())){ return m; diff --git a/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/query/QueryCondition.java b/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/query/QueryCondition.java new file mode 100644 index 000000000..6a988a19d --- /dev/null +++ b/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/query/QueryCondition.java @@ -0,0 +1,122 @@ +/** + * Copyright 2019 The JoyQueue Authors. + * + * Licensed 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.joyqueue.server.archive.store.query; + +import org.joyqueue.server.archive.store.model.Query; +import org.joyqueue.server.archive.store.utils.ArchiveSerializer; + +/** + * Created by chengzhiliang on 2018/12/4. + */ +public class QueryCondition implements Query { + + @Override + public T getQueryCondition() { + return (T)this; + } + + private RowKey startRowKey; // 查询开始键 + private RowKey stopRowKey; // 查询结束键 + private int count; + + private RowKey rowKey; // 指定RowKey查询 + + private byte[] startRowKeyByteArr; + + public int getCount() { + return count; + } + + public void setCount(int count) { + this.count = count; + } + + public RowKey getStartRowKey() { + return startRowKey; + } + + public void setStartRowKey(RowKey startRowKey) { + this.startRowKey = startRowKey; + } + + public RowKey getStopRowKey() { + return stopRowKey; + } + + public void setStopRowKey(RowKey stopRowKey) { + this.stopRowKey = stopRowKey; + } + + public RowKey getRowKey() { + return rowKey; + } + + public void setRowKey(RowKey rowKey) { + this.rowKey = rowKey; + } + + public byte[] getStartRowKeyByteArr() { + return startRowKeyByteArr; + } + + public void setStartRowKeyByteArr(String startRowKeyByteArr) { + byte[] bytes = ArchiveSerializer.hexStrToByteArray(startRowKeyByteArr); + this.startRowKeyByteArr = bytes; + } + + /** + * 查询RowKey + */ + public static class RowKey { + private String topic; + private long time; + private String businessId; + private String messageId; + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public long getTime() { + return time; + } + + public void setTime(long time) { + this.time = time; + } + + public String getBusinessId() { + return businessId; + } + + public void setBusinessId(String businessId) { + this.businessId = businessId; + } + + public String getMessageId() { + return messageId; + } + + public void setMessageId(String messageId) { + this.messageId = messageId; + } + + } +} diff --git a/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/utils/ArchiveSerializer.java b/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/utils/ArchiveSerializer.java new file mode 100644 index 000000000..9dd74dac9 --- /dev/null +++ b/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/utils/ArchiveSerializer.java @@ -0,0 +1,264 @@ +package org.joyqueue.server.archive.store.utils; + +import org.joyqueue.server.archive.store.model.ConsumeLog; +import org.joyqueue.server.archive.store.model.SendLog; +import org.joyqueue.toolkit.lang.Pair; +import org.joyqueue.toolkit.security.Md5; + +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.security.GeneralSecurityException; + +public class ArchiveSerializer { + + public static ConsumeLog readConsumeLog(Pair pair) { + ConsumeLog log = new ConsumeLog(); + + byte[] key = pair.getKey(); + ByteBuffer wrap = ByteBuffer.wrap(key); + + byte[] messageId = new byte[16]; + wrap.get(messageId); + log.setBytesMessageId(messageId); + + int appId = wrap.getInt(); + log.setAppId(appId); + + byte[] value = pair.getValue(); + ByteBuffer valBF = ByteBuffer.wrap(value); + + log.setBrokerId(valBF.getInt()); + + byte[] clientIp = new byte[16]; + valBF.get(clientIp); + log.setClientIp(clientIp); + + log.setConsumeTime(valBF.getLong()); + + return log; + } + + public static Pair convertConsumeLogToKVBytes(ConsumeLog consumeLog) { + ByteBuffer buffer = ByteBuffer.allocate(ConsumeLog.len); + buffer.put(consumeLog.getBytesMessageId()); + buffer.putInt(consumeLog.getAppId()); + buffer.putInt(consumeLog.getBrokerId()); + + byte[] clientIpBytes16 = new byte[16]; + byte[] clientIpBytes = consumeLog.getClientIp(); + System.arraycopy(clientIpBytes, 0, clientIpBytes16,0, Math.min(clientIpBytes.length, clientIpBytes16.length)); + buffer.put(clientIpBytes16); + + buffer.putLong(consumeLog.getConsumeTime()); + buffer.flip(); + + byte[] key = new byte[ConsumeLog.keyLen]; + buffer.get(key); + + byte[] value = new byte[ConsumeLog.valLen]; + buffer.get(value); + + return new Pair<>(key, value); + } + + /** + * key: topicId(4) + sendTime(8) + businessId(16) + messageId(16) 总长度:44 + * value: brokerId(4) + appId(4) + clientIp(16) + sendTime(8) + compassType(2) + messageBody(变长) + businessId(变长) + * + * @param sendLog + * @return + */ + public static Pair convertSendLogToKVBytes(SendLog sendLog) throws GeneralSecurityException { + ByteBuffer bufferKey = ByteBuffer.allocate(44); + bufferKey.putInt(sendLog.getTopicId()); + bufferKey.putLong(sendLog.getSendTime()); + bufferKey.put(Md5.INSTANCE.encrypt(sendLog.getBusinessId().getBytes(Charset.forName("utf-8")), null)); + bufferKey.put(md5(sendLog.getMessageId(),null)); + + + // value + byte[] messageBody = sendLog.getMessageBody(); + byte[] businessIdBytes = sendLog.getBusinessId().getBytes(Charset.forName("utf-8")); + int size = 4 + 4 + 16 + 8 + 2 + 4 + messageBody.length + 4 + businessIdBytes.length; + ByteBuffer bufferVal = ByteBuffer.allocate(size); + bufferVal.putInt(sendLog.getBrokerId()); + bufferVal.putInt(sendLog.getAppId()); + + // clientIP + byte[] clientIpBytes16 = new byte[16]; + byte[] clientIpBytes = sendLog.getClientIp(); + System.arraycopy(clientIpBytes, 0, clientIpBytes16,0, Math.min(clientIpBytes.length, clientIpBytes16.length)); + bufferVal.put(clientIpBytes16); + + bufferVal.putShort(sendLog.getCompressType()); + bufferVal.putInt(messageBody.length); + bufferVal.put(messageBody); + bufferVal.putInt(businessIdBytes.length); + bufferVal.put(businessIdBytes); + + return new Pair<>(bufferKey.array(), bufferVal.array()); + } + + /** + * key: topicId(4) + businessId(16) + sendTime(8) + messageId(16) 总长度:44 + * value: brokerId(4) + appId(4) + clientIp(16) + sendTime(8) + compassType(2) + messageBody(变长) + businessId(变长) + * + * @param sendLog + * @return + */ + public static Pair convertSendLogToKVBytes4BizId(SendLog sendLog) throws GeneralSecurityException { + ByteBuffer bufferKey = ByteBuffer.allocate(44); + bufferKey.putInt(sendLog.getTopicId()); + bufferKey.put(Md5.INSTANCE.encrypt(sendLog.getBusinessId().getBytes(Charset.forName("utf-8")), null)); + bufferKey.putLong(sendLog.getSendTime()); + bufferKey.put(md5(sendLog.getMessageId(), null)); + + + // value + byte[] messageBody = sendLog.getMessageBody(); + byte[] businessIdBytes = sendLog.getBusinessId().getBytes(Charset.forName("utf-8")); + int size = 4 + 4 + 16 + 8 + 2 + 4 + messageBody.length + 4 + businessIdBytes.length; + ByteBuffer bufferVal = ByteBuffer.allocate(size); + bufferVal.putInt(sendLog.getBrokerId()); + bufferVal.putInt(sendLog.getAppId()); + + // clientIP + byte[] clientIpBytes16 = new byte[16]; + byte[] clientIpBytes = sendLog.getClientIp(); + System.arraycopy(clientIpBytes, 0, clientIpBytes16,0, Math.min(clientIpBytes.length, clientIpBytes16.length)); + bufferVal.put(clientIpBytes16); + + bufferVal.putShort(sendLog.getCompressType()); + bufferVal.putInt(messageBody.length); + bufferVal.put(messageBody); + bufferVal.putInt(businessIdBytes.length); + bufferVal.put(businessIdBytes); + + return new Pair<>(bufferKey.array(), bufferVal.array()); + } + + public static SendLog readSendLog(Pair pair) { + SendLog log = new SendLog(); + + byte[] key = pair.getKey(); + ByteBuffer wrap = ByteBuffer.wrap(key); + // 主题ID + log.setTopicId(wrap.getInt()); + // 发送时间 + log.setSendTime(wrap.getLong()); + // 业务主键(MD5后的) + byte[] businessId = new byte[16]; + wrap.get(businessId); + // 消息ID(MD5后的) + byte[] messageId = new byte[16]; + wrap.get(messageId); + log.setBytesMessageId(messageId); + log.setMessageId(byteArrayToHexStr(messageId)); + + byte[] value = pair.getValue(); + ByteBuffer valWrap = ByteBuffer.wrap(value); + // brokerID + log.setBrokerId(valWrap.getInt()); + // 应用ID + log.setAppId(valWrap.getInt()); + // 客户端IP + byte[] clientIp = new byte[16]; + valWrap.get(clientIp); + log.setClientIp(clientIp); + // 压缩类型 + log.setCompressType(valWrap.getShort()); + // 消息体 + int msgBodySize = valWrap.getInt(); + byte[] messageBody = new byte[msgBodySize]; + valWrap.get(messageBody); + log.setMessageBody(messageBody); + // 业务主键 + int bizSize = valWrap.getInt(); + byte[] businessIdBytes = new byte[bizSize]; + valWrap.get(businessIdBytes); + log.setBusinessId(new String(businessIdBytes, Charset.forName("utf-8"))); + + return log; + } + + public static SendLog readSendLog4BizId(Pair pair) { + SendLog log = new SendLog(); + + byte[] key = pair.getKey(); + ByteBuffer wrap = ByteBuffer.wrap(key); + // 主题ID + log.setTopicId(wrap.getInt()); + // 业务主键(MD5后的) + byte[] businessId = new byte[16]; + wrap.get(businessId); + // 发送时间 + log.setSendTime(wrap.getLong()); + // 消息ID(MD5后的) + byte[] messageId = new byte[16]; + wrap.get(messageId); + log.setBytesMessageId(messageId); + log.setMessageId(byteArrayToHexStr(messageId)); + + byte[] value = pair.getValue(); + ByteBuffer valWrap = ByteBuffer.wrap(value); + // brokerID + log.setBrokerId(valWrap.getInt()); + // 应用ID + log.setAppId(valWrap.getInt()); + // 客户端IP + byte[] clientIp = new byte[16]; + valWrap.get(clientIp); + log.setClientIp(clientIp); + // 压缩类型 + log.setCompressType(valWrap.getShort()); + // 消息体 + int msgBodySize = valWrap.getInt(); + byte[] messageBody = new byte[msgBodySize]; + valWrap.get(messageBody); + log.setMessageBody(messageBody); + // 业务主键 + int bizSize = valWrap.getInt(); + byte[] businessIdBytes = new byte[bizSize]; + valWrap.get(businessIdBytes); + log.setBusinessId(new String(businessIdBytes, Charset.forName("utf-8"))); + + return log; + } + + /** + * MD5 for content with key + * + **/ + public static byte[] md5(String content,byte[] key) throws GeneralSecurityException { + return Md5.INSTANCE.encrypt(content.getBytes(Charset.forName("utf-8")), key); + } + + public static String byteArrayToHexStr(byte[] byteArray) { + if (byteArray == null){ + return null; + } + char[] hexArray = "0123456789ABCDEF".toCharArray(); + char[] hexChars = new char[byteArray.length * 2]; + for (int j = 0; j < byteArray.length; j++) { + int v = byteArray[j] & 0xFF; + hexChars[j * 2] = hexArray[v >>> 4]; + hexChars[j * 2 + 1] = hexArray[v & 0x0F]; + } + return new String(hexChars); + } + + public static byte[] hexStrToByteArray(String str) { + if (str == null) { + return null; + } + if (str.length() == 0) { + return new byte[0]; + } + byte[] byteArray = new byte[str.length() / 2]; + for (int i = 0; i < byteArray.length; i++){ + String subStr = str.substring(2 * i, 2 * i + 2); + byteArray[i] = ((byte)Integer.parseInt(subStr, 16)); + } + return byteArray; + } +} From 2928858825a0dec02268bf02edabc77dce241937 Mon Sep 17 00:00:00 2001 From: majun87 Date: Tue, 15 Dec 2020 20:10:45 +0800 Subject: [PATCH 06/15] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E5=BC=80=E6=BA=90?= =?UTF-8?q?=E5=AE=9E=E7=8E=B0=E4=BE=9D=E8=B5=96=EF=BC=8C=E5=89=94=E9=99=A4?= =?UTF-8?q?=E5=8C=85=E6=8B=ACdata-service=E4=BE=9D=E8=B5=96=E7=9A=84?= =?UTF-8?q?=E7=9B=B8=E5=85=B3=E5=AE=9E=E7=8E=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../toolkit/serialize/AbstractSerializer.java | 12 ++++++++++++ .../joyqueue-data/joyqueue-data-service/pom.xml | 6 +++++- .../nsr/impl/ConsumerNameServerServiceImpl.java | 2 +- .../routing/command/archive/ArchiveCommand.java | 5 ++--- .../routing/command/broker/BrokerCommand.java | 2 +- pom.xml | 6 ++++++ 6 files changed, 27 insertions(+), 6 deletions(-) diff --git a/joyqueue-common/joyqueue-toolkit/src/main/java/org/joyqueue/toolkit/serialize/AbstractSerializer.java b/joyqueue-common/joyqueue-toolkit/src/main/java/org/joyqueue/toolkit/serialize/AbstractSerializer.java index 38dfc0e23..92ceb261f 100644 --- a/joyqueue-common/joyqueue-toolkit/src/main/java/org/joyqueue/toolkit/serialize/AbstractSerializer.java +++ b/joyqueue-common/joyqueue-toolkit/src/main/java/org/joyqueue/toolkit/serialize/AbstractSerializer.java @@ -229,6 +229,18 @@ public static Map toStringMap(final String text) throws IOExcept return new HashMap(properties); } + public static String readString(final byte[] bytes) { + return bytes == null ? null : readString(bytes, 0, bytes.length); + } + + public static String readString(byte[] bytes, int offset, int length) { + if (bytes == null) { + return null; + } else { + return length == 0 ? "" : new String(bytes, offset, length, Charset.forName("UTF-8")); + } + } + /** * 读取字符串,字符长度<=255 * diff --git a/joyqueue-console/joyqueue-data/joyqueue-data-service/pom.xml b/joyqueue-console/joyqueue-data/joyqueue-data-service/pom.xml index f2661cafb..aff86852f 100644 --- a/joyqueue-console/joyqueue-data/joyqueue-data-service/pom.xml +++ b/joyqueue-console/joyqueue-data/joyqueue-data-service/pom.xml @@ -112,9 +112,13 @@ commons-beanutils + commons-net + commons-net + + org.joyqueue joyqueue-nsr-core diff --git a/joyqueue-console/joyqueue-data/joyqueue-data-service/src/main/java/org/joyqueue/nsr/impl/ConsumerNameServerServiceImpl.java b/joyqueue-console/joyqueue-data/joyqueue-data-service/src/main/java/org/joyqueue/nsr/impl/ConsumerNameServerServiceImpl.java index e1dd9b1a2..60c39ec39 100644 --- a/joyqueue-console/joyqueue-data/joyqueue-data-service/src/main/java/org/joyqueue/nsr/impl/ConsumerNameServerServiceImpl.java +++ b/joyqueue-console/joyqueue-data/joyqueue-data-service/src/main/java/org/joyqueue/nsr/impl/ConsumerNameServerServiceImpl.java @@ -17,6 +17,7 @@ import com.alibaba.fastjson.JSON; import com.alibaba.fastjson.JSONObject; +import org.apache.commons.lang3.StringUtils; import org.joyqueue.convert.CodeConverter; import org.joyqueue.convert.NsrConsumerConverter; import org.joyqueue.domain.ClientType; @@ -26,7 +27,6 @@ import org.joyqueue.nsr.ConsumerNameServerService; import org.joyqueue.nsr.NameServerBase; import org.joyqueue.nsr.model.ConsumerQuery; -import org.apache.commons.lang.StringUtils; import org.springframework.stereotype.Service; import java.util.List; diff --git a/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/archive/ArchiveCommand.java b/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/archive/ArchiveCommand.java index 9b8194ad7..98486e26e 100644 --- a/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/archive/ArchiveCommand.java +++ b/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/archive/ArchiveCommand.java @@ -17,7 +17,7 @@ import com.alibaba.fastjson.JSON; import org.joyqueue.broker.archive.ArchiveUtils; -import org.joyqueue.broker.buffer.Serializer; +import org.joyqueue.util.serializer.Serializer; import org.joyqueue.exception.ServiceException; import org.joyqueue.handler.error.ErrorCode; import org.joyqueue.message.SourceType; @@ -45,7 +45,6 @@ import com.jd.laf.web.vertx.response.Responses; import io.vertx.core.http.HttpServerRequest; import io.vertx.core.http.HttpServerResponse; -import org.apache.hadoop.hbase.util.Bytes; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; @@ -221,7 +220,7 @@ public String preview(BrokerMessage brokerMessage,String messageType){ return messagePreviewService.preview(messageType, brokerMessage.getDecompressedBody()); } catch (Throwable e) { logger.error("parse error",e); - return Bytes.toString(brokerMessage.getDecompressedBody()); + return Serializer.readString(brokerMessage.getDecompressedBody()); } } diff --git a/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/broker/BrokerCommand.java b/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/broker/BrokerCommand.java index 8bd6b2490..f324a2178 100644 --- a/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/broker/BrokerCommand.java +++ b/joyqueue-console/joyqueue-web/joyqueue-web-handler/src/main/java/org/joyqueue/handler/routing/command/broker/BrokerCommand.java @@ -21,6 +21,7 @@ import com.jd.laf.web.vertx.annotation.QueryParam; import com.jd.laf.web.vertx.response.Response; import com.jd.laf.web.vertx.response.Responses; +import org.apache.commons.net.telnet.TelnetClient; import org.joyqueue.handler.annotation.PageQuery; import org.joyqueue.handler.error.ConfigException; import org.joyqueue.handler.routing.command.NsrCommandSupport; @@ -29,7 +30,6 @@ import org.joyqueue.model.domain.Broker; import org.joyqueue.model.query.QBroker; import org.joyqueue.service.BrokerService; -import org.apache.commons.net.telnet.TelnetClient; import static org.joyqueue.handler.Constants.ID; diff --git a/pom.xml b/pom.xml index 616faacfb..d2d4389d1 100644 --- a/pom.xml +++ b/pom.xml @@ -78,6 +78,7 @@ 3.2.2 4.1 1.4 + 3.1 4.5.12 4.1.4 2.9.10.4 @@ -580,6 +581,11 @@ commons-beanutils ${commons-beanutils.version} + + commons-net + commons-net + ${commons-net.version} + commons-logging commons-logging From 9fbd8c4e8a0fa08977d6768c1a6a05c35412a981 Mon Sep 17 00:00:00 2001 From: majun87 Date: Wed, 16 Dec 2020 21:43:24 +0800 Subject: [PATCH 07/15] =?UTF-8?q?=E8=B0=83=E6=95=B4=E5=BC=80=E6=BA=90hbase?= =?UTF-8?q?=E5=AE=9E=E7=8E=B0=E7=B1=BB=E4=BE=9D=E8=B5=96?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../archive/store/query/QueryCondition.java | 22 ++ .../server/archive/store/HBaseSerializer.java | 281 ------------------ .../server/archive/store/HBaseStore.java | 24 +- .../server/archive/store/QueryCondition.java | 121 -------- 4 files changed, 35 insertions(+), 413 deletions(-) delete mode 100644 joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/HBaseSerializer.java delete mode 100644 joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/QueryCondition.java diff --git a/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/query/QueryCondition.java b/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/query/QueryCondition.java index 6a988a19d..4e8e99927 100644 --- a/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/query/QueryCondition.java +++ b/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/query/QueryCondition.java @@ -18,6 +18,8 @@ import org.joyqueue.server.archive.store.model.Query; import org.joyqueue.server.archive.store.utils.ArchiveSerializer; +import java.util.Arrays; + /** * Created by chengzhiliang on 2018/12/4. */ @@ -77,6 +79,17 @@ public void setStartRowKeyByteArr(String startRowKeyByteArr) { this.startRowKeyByteArr = bytes; } + @Override + public String toString() { + return "QueryCondition{" + + "startRowKey=" + startRowKey + + ", stopRowKey=" + stopRowKey + + ", count=" + count + + ", rowKey=" + rowKey + + ", startRowKeyByteArr=" + Arrays.toString(startRowKeyByteArr) + + '}'; + } + /** * 查询RowKey */ @@ -118,5 +131,14 @@ public void setMessageId(String messageId) { this.messageId = messageId; } + @Override + public String toString() { + return "RowKey{" + + "topic='" + topic + '\'' + + ", time=" + time + + ", businessId='" + businessId + '\'' + + ", messageId='" + messageId + '\'' + + '}'; + } } } diff --git a/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/HBaseSerializer.java b/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/HBaseSerializer.java deleted file mode 100644 index d70ca96e6..000000000 --- a/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/HBaseSerializer.java +++ /dev/null @@ -1,281 +0,0 @@ -/** - * Copyright 2019 The JoyQueue Authors. - * - * Licensed 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.joyqueue.server.archive.store; - -import org.joyqueue.server.archive.store.model.ConsumeLog; -import org.joyqueue.server.archive.store.model.SendLog; -import org.joyqueue.toolkit.lang.Pair; -import org.joyqueue.toolkit.security.Md5; -import java.nio.ByteBuffer; -import java.nio.charset.Charset; -import java.security.GeneralSecurityException; - -/** - * Created by chengzhiliang on 2018/12/12. - */ -public class HBaseSerializer { - - public static ConsumeLog readConsumeLog(Pair pair) { - ConsumeLog log = new ConsumeLog(); - - byte[] key = pair.getKey(); - ByteBuffer wrap = ByteBuffer.wrap(key); - - byte[] messageId = new byte[16]; - wrap.get(messageId); - log.setBytesMessageId(messageId); - - int appId = wrap.getInt(); - log.setAppId(appId); - - byte[] value = pair.getValue(); - ByteBuffer valBF = ByteBuffer.wrap(value); - - log.setBrokerId(valBF.getInt()); - - byte[] clientIp = new byte[16]; - valBF.get(clientIp); - log.setClientIp(clientIp); - - log.setConsumeTime(valBF.getLong()); - - return log; - } - - public static Pair convertConsumeLogToKVBytes(ConsumeLog consumeLog) { - ByteBuffer buffer = ByteBuffer.allocate(ConsumeLog.len); - buffer.put(consumeLog.getBytesMessageId()); - buffer.putInt(consumeLog.getAppId()); - buffer.putInt(consumeLog.getBrokerId()); - - byte[] clientIpBytes16 = new byte[16]; - byte[] clientIpBytes = consumeLog.getClientIp(); - System.arraycopy(clientIpBytes, 0, clientIpBytes16,0, Math.min(clientIpBytes.length, clientIpBytes16.length)); - buffer.put(clientIpBytes16); - - buffer.putLong(consumeLog.getConsumeTime()); - buffer.flip(); - - byte[] key = new byte[ConsumeLog.keyLen]; - buffer.get(key); - - byte[] value = new byte[ConsumeLog.valLen]; - buffer.get(value); - - return new Pair<>(key, value); - } - - /** - * key: topicId(4) + sendTime(8) + businessId(16) + messageId(16) 总长度:44 - * value: brokerId(4) + appId(4) + clientIp(16) + sendTime(8) + compassType(2) + messageBody(变长) + businessId(变长) - * - * @param sendLog - * @return - */ - public static Pair convertSendLogToKVBytes(SendLog sendLog) throws GeneralSecurityException { - ByteBuffer bufferKey = ByteBuffer.allocate(44); - bufferKey.putInt(sendLog.getTopicId()); - bufferKey.putLong(sendLog.getSendTime()); - bufferKey.put(Md5.INSTANCE.encrypt(sendLog.getBusinessId().getBytes(Charset.forName("utf-8")), null)); - bufferKey.put(md5(sendLog.getMessageId(),null)); - - - // value - byte[] messageBody = sendLog.getMessageBody(); - byte[] businessIdBytes = sendLog.getBusinessId().getBytes(Charset.forName("utf-8")); - int size = 4 + 4 + 16 + 8 + 2 + 4 + messageBody.length + 4 + businessIdBytes.length; - ByteBuffer bufferVal = ByteBuffer.allocate(size); - bufferVal.putInt(sendLog.getBrokerId()); - bufferVal.putInt(sendLog.getAppId()); - - // clientIP - byte[] clientIpBytes16 = new byte[16]; - byte[] clientIpBytes = sendLog.getClientIp(); - System.arraycopy(clientIpBytes, 0, clientIpBytes16,0, Math.min(clientIpBytes.length, clientIpBytes16.length)); - bufferVal.put(clientIpBytes16); - - bufferVal.putShort(sendLog.getCompressType()); - bufferVal.putInt(messageBody.length); - bufferVal.put(messageBody); - bufferVal.putInt(businessIdBytes.length); - bufferVal.put(businessIdBytes); - - return new Pair<>(bufferKey.array(), bufferVal.array()); - } - - /** - * key: topicId(4) + businessId(16) + sendTime(8) + messageId(16) 总长度:44 - * value: brokerId(4) + appId(4) + clientIp(16) + sendTime(8) + compassType(2) + messageBody(变长) + businessId(变长) - * - * @param sendLog - * @return - */ - public static Pair convertSendLogToKVBytes4BizId(SendLog sendLog) throws GeneralSecurityException { - ByteBuffer bufferKey = ByteBuffer.allocate(44); - bufferKey.putInt(sendLog.getTopicId()); - bufferKey.put(Md5.INSTANCE.encrypt(sendLog.getBusinessId().getBytes(Charset.forName("utf-8")), null)); - bufferKey.putLong(sendLog.getSendTime()); - bufferKey.put(md5(sendLog.getMessageId(), null)); - - - // value - byte[] messageBody = sendLog.getMessageBody(); - byte[] businessIdBytes = sendLog.getBusinessId().getBytes(Charset.forName("utf-8")); - int size = 4 + 4 + 16 + 8 + 2 + 4 + messageBody.length + 4 + businessIdBytes.length; - ByteBuffer bufferVal = ByteBuffer.allocate(size); - bufferVal.putInt(sendLog.getBrokerId()); - bufferVal.putInt(sendLog.getAppId()); - - // clientIP - byte[] clientIpBytes16 = new byte[16]; - byte[] clientIpBytes = sendLog.getClientIp(); - System.arraycopy(clientIpBytes, 0, clientIpBytes16,0, Math.min(clientIpBytes.length, clientIpBytes16.length)); - bufferVal.put(clientIpBytes16); - - bufferVal.putShort(sendLog.getCompressType()); - bufferVal.putInt(messageBody.length); - bufferVal.put(messageBody); - bufferVal.putInt(businessIdBytes.length); - bufferVal.put(businessIdBytes); - - return new Pair<>(bufferKey.array(), bufferVal.array()); - } - - public static SendLog readSendLog(Pair pair) { - SendLog log = new SendLog(); - - byte[] key = pair.getKey(); - ByteBuffer wrap = ByteBuffer.wrap(key); - // 主题ID - log.setTopicId(wrap.getInt()); - // 发送时间 - log.setSendTime(wrap.getLong()); - // 业务主键(MD5后的) - byte[] businessId = new byte[16]; - wrap.get(businessId); - // 消息ID(MD5后的) - byte[] messageId = new byte[16]; - wrap.get(messageId); - log.setBytesMessageId(messageId); - log.setMessageId(byteArrayToHexStr(messageId)); - - byte[] value = pair.getValue(); - ByteBuffer valWrap = ByteBuffer.wrap(value); - // brokerID - log.setBrokerId(valWrap.getInt()); - // 应用ID - log.setAppId(valWrap.getInt()); - // 客户端IP - byte[] clientIp = new byte[16]; - valWrap.get(clientIp); - log.setClientIp(clientIp); - // 压缩类型 - log.setCompressType(valWrap.getShort()); - // 消息体 - int msgBodySize = valWrap.getInt(); - byte[] messageBody = new byte[msgBodySize]; - valWrap.get(messageBody); - log.setMessageBody(messageBody); - // 业务主键 - int bizSize = valWrap.getInt(); - byte[] businessIdBytes = new byte[bizSize]; - valWrap.get(businessIdBytes); - log.setBusinessId(new String(businessIdBytes, Charset.forName("utf-8"))); - - return log; - } - - public static SendLog readSendLog4BizId(Pair pair) { - SendLog log = new SendLog(); - - byte[] key = pair.getKey(); - ByteBuffer wrap = ByteBuffer.wrap(key); - // 主题ID - log.setTopicId(wrap.getInt()); - // 业务主键(MD5后的) - byte[] businessId = new byte[16]; - wrap.get(businessId); - // 发送时间 - log.setSendTime(wrap.getLong()); - // 消息ID(MD5后的) - byte[] messageId = new byte[16]; - wrap.get(messageId); - log.setBytesMessageId(messageId); - log.setMessageId(byteArrayToHexStr(messageId)); - - byte[] value = pair.getValue(); - ByteBuffer valWrap = ByteBuffer.wrap(value); - // brokerID - log.setBrokerId(valWrap.getInt()); - // 应用ID - log.setAppId(valWrap.getInt()); - // 客户端IP - byte[] clientIp = new byte[16]; - valWrap.get(clientIp); - log.setClientIp(clientIp); - // 压缩类型 - log.setCompressType(valWrap.getShort()); - // 消息体 - int msgBodySize = valWrap.getInt(); - byte[] messageBody = new byte[msgBodySize]; - valWrap.get(messageBody); - log.setMessageBody(messageBody); - // 业务主键 - int bizSize = valWrap.getInt(); - byte[] businessIdBytes = new byte[bizSize]; - valWrap.get(businessIdBytes); - log.setBusinessId(new String(businessIdBytes, Charset.forName("utf-8"))); - - return log; - } - - /** - * MD5 for content with key - * - **/ - public static byte[] md5(String content,byte[] key) throws GeneralSecurityException { - return Md5.INSTANCE.encrypt(content.getBytes(Charset.forName("utf-8")), key); - } - - public static String byteArrayToHexStr(byte[] byteArray) { - if (byteArray == null){ - return null; - } - char[] hexArray = "0123456789ABCDEF".toCharArray(); - char[] hexChars = new char[byteArray.length * 2]; - for (int j = 0; j < byteArray.length; j++) { - int v = byteArray[j] & 0xFF; - hexChars[j * 2] = hexArray[v >>> 4]; - hexChars[j * 2 + 1] = hexArray[v & 0x0F]; - } - return new String(hexChars); - } - - public static byte[] hexStrToByteArray(String str) { - if (str == null) { - return null; - } - if (str.length() == 0) { - return new byte[0]; - } - byte[] byteArray = new byte[str.length() / 2]; - for (int i = 0; i < byteArray.length; i++){ - String subStr = str.substring(2 * i, 2 * i + 2); - byteArray[i] = ((byte)Integer.parseInt(subStr, 16)); - } - return byteArray; - } -} diff --git a/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/HBaseStore.java b/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/HBaseStore.java index fa9cc9d88..2d56baafa 100644 --- a/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/HBaseStore.java +++ b/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/HBaseStore.java @@ -22,6 +22,8 @@ import org.joyqueue.server.archive.store.api.ArchiveStore; import org.joyqueue.monitor.PointTracer; import org.joyqueue.server.archive.store.model.*; +import org.joyqueue.server.archive.store.query.QueryCondition; +import org.joyqueue.server.archive.store.utils.ArchiveSerializer; import org.joyqueue.toolkit.lang.Pair; import org.joyqueue.toolkit.network.IpUtil; import org.joyqueue.toolkit.security.Md5; @@ -118,7 +120,7 @@ public void putConsumeLog(List consumeLogList, PointTracer tracer) t int appId = topicAppMapping.getAppId(app); consumeLog.setAppId(appId); - Pair pair = HBaseSerializer.convertConsumeLogToKVBytes(consumeLog); + Pair pair = ArchiveSerializer.convertConsumeLogToKVBytes(consumeLog); logList.add(pair); } @@ -145,10 +147,10 @@ public void putSendLog(List sendLogList, PointTracer tracer) throws Joy log.setTopicId(topicId); log.setAppId(appId); - Pair pair = HBaseSerializer.convertSendLogToKVBytes(log); + Pair pair = ArchiveSerializer.convertSendLogToKVBytes(log); logList.add(pair); - Pair pair4BizId = HBaseSerializer.convertSendLogToKVBytes4BizId(log); + Pair pair4BizId = ArchiveSerializer.convertSendLogToKVBytes4BizId(log); logList.add(pair4BizId); } @@ -233,13 +235,13 @@ public List scanSendLog(Query query) throws JoyQueueException { for (Pair pair : scan) { SendLog log; if (hasBizId) { - log = HBaseSerializer.readSendLog4BizId(pair); + log = ArchiveSerializer.readSendLog4BizId(pair); } else { - log = HBaseSerializer.readSendLog(pair); + log = ArchiveSerializer.readSendLog(pair); } log.setClientIpStr(toIpString(log.getClientIp())); - log.setRowKeyStart(HBaseSerializer.byteArrayToHexStr(pair.getKey())); + log.setRowKeyStart(ArchiveSerializer.byteArrayToHexStr(pair.getKey())); String topicName = topicAppMapping.getTopicName(log.getTopicId()); log.setTopic(topicName); @@ -412,13 +414,13 @@ public SendLog getOneSendLog(Query query) throws JoyQueueException { allocate.putInt(topicAppMapping.getTopicId(rowKey.getTopic())); allocate.putLong(rowKey.getTime()); allocate.put(Md5.INSTANCE.encrypt(rowKey.getBusinessId().getBytes(Charset.forName("utf-8")), null)); - allocate.put(HBaseSerializer.hexStrToByteArray(rowKey.getMessageId())); + allocate.put(ArchiveSerializer.hexStrToByteArray(rowKey.getMessageId())); // rowKey byte[] bytesRowKey = allocate.array(); Pair bytes = hBaseClient.getKV(namespace, sendLogTable, cf, col, bytesRowKey); - SendLog log = HBaseSerializer.readSendLog(bytes); + SendLog log = ArchiveSerializer.readSendLog(bytes); StringBuilder clientIp = new StringBuilder(); IpUtil.toAddress(log.getClientIp(), clientIp); @@ -452,7 +454,7 @@ public List scanConsumeLog(String messageId, Integer count) throws J scanParameters.setCf(cf); scanParameters.setCol(col); - byte[] messageIdBytes = HBaseSerializer.hexStrToByteArray(messageId); + byte[] messageIdBytes = ArchiveSerializer.hexStrToByteArray(messageId); scanParameters.setStartRowKey(messageIdBytes); ByteBuffer bytebuffer = ByteBuffer.allocate(messageIdBytes.length + 1); @@ -465,8 +467,8 @@ public List scanConsumeLog(String messageId, Integer count) throws J List> scan = hBaseClient.scan(namespace, scanParameters); for (Pair pair : scan) { - ConsumeLog log = HBaseSerializer.readConsumeLog(pair); - log.setMessageId(HBaseSerializer.byteArrayToHexStr(log.getBytesMessageId())); + ConsumeLog log = ArchiveSerializer.readConsumeLog(pair); + log.setMessageId(ArchiveSerializer.byteArrayToHexStr(log.getBytesMessageId())); StringBuilder clientIp = new StringBuilder(); IpUtil.toAddress(log.getClientIp(), clientIp); diff --git a/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/QueryCondition.java b/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/QueryCondition.java deleted file mode 100644 index 8ed2a420a..000000000 --- a/joyqueue-server/joyqueue-archive/joyqueue-archive-hbase/src/main/java/org/joyqueue/server/archive/store/QueryCondition.java +++ /dev/null @@ -1,121 +0,0 @@ -/** - * Copyright 2019 The JoyQueue Authors. - * - * Licensed 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.joyqueue.server.archive.store; - -import org.joyqueue.server.archive.store.model.Query; - -/** - * Created by chengzhiliang on 2018/12/4. - */ -public class QueryCondition implements Query { - - @Override - public T getQueryCondition() { - return (T)this; - } - - private RowKey startRowKey; // 查询开始键 - private RowKey stopRowKey; // 查询结束键 - private int count; - - private RowKey rowKey; // 指定RowKey查询 - - private byte[] startRowKeyByteArr; - - public int getCount() { - return count; - } - - public void setCount(int count) { - this.count = count; - } - - public RowKey getStartRowKey() { - return startRowKey; - } - - public void setStartRowKey(RowKey startRowKey) { - this.startRowKey = startRowKey; - } - - public RowKey getStopRowKey() { - return stopRowKey; - } - - public void setStopRowKey(RowKey stopRowKey) { - this.stopRowKey = stopRowKey; - } - - public RowKey getRowKey() { - return rowKey; - } - - public void setRowKey(RowKey rowKey) { - this.rowKey = rowKey; - } - - public byte[] getStartRowKeyByteArr() { - return startRowKeyByteArr; - } - - public void setStartRowKeyByteArr(String startRowKeyByteArr) { - byte[] bytes = HBaseSerializer.hexStrToByteArray(startRowKeyByteArr); - this.startRowKeyByteArr = bytes; - } - - /** - * 查询RowKey - */ - public static class RowKey { - private String topic; - private long time; - private String businessId; - private String messageId; - - public String getTopic() { - return topic; - } - - public void setTopic(String topic) { - this.topic = topic; - } - - public long getTime() { - return time; - } - - public void setTime(long time) { - this.time = time; - } - - public String getBusinessId() { - return businessId; - } - - public void setBusinessId(String businessId) { - this.businessId = businessId; - } - - public String getMessageId() { - return messageId; - } - - public void setMessageId(String messageId) { - this.messageId = messageId; - } - - } -} From 4672d920cb228628fdcc68ce45c09cc9ba4c14ec Mon Sep 17 00:00:00 2001 From: majun87 Date: Thu, 17 Dec 2020 21:08:40 +0800 Subject: [PATCH 08/15] =?UTF-8?q?=E7=AE=80=E5=8D=95=E6=B6=88=E8=B4=B9?= =?UTF-8?q?=E5=BD=92=E6=A1=A3=E9=99=90=E6=B5=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../broker/archive/ArchiveManager.java | 11 ++++- .../broker/archive/ConsumeArchiveService.java | 47 ++++++++++++++++--- .../broker/archive/ProduceArchiveService.java | 27 ++++++++++- 3 files changed, 74 insertions(+), 11 deletions(-) diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveManager.java index 1b2bbfd99..03206e5c0 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveManager.java @@ -16,6 +16,7 @@ package org.joyqueue.broker.archive; import org.joyqueue.broker.BrokerContext; +import org.joyqueue.broker.limit.SubscribeRateLimiter; import org.joyqueue.toolkit.lang.Close; import org.joyqueue.toolkit.service.Service; import org.slf4j.Logger; @@ -41,6 +42,9 @@ public class ArchiveManager extends Service { // 归档配置 private ArchiveConfig archiveConfig; + // 归档限流器 + private SubscribeRateLimiter rateLimiterManager; + public ArchiveManager(BrokerContext context) { this.context = context; } @@ -73,11 +77,14 @@ protected void validate() throws Exception { if (archiveConfig == null) { archiveConfig = new ArchiveConfig(context == null ? null : context.getPropertySupplier()); } + if (rateLimiterManager == null) { + rateLimiterManager = new ArchiveRateLimiterManager(context); + } if (sendArchiveService == null) { - this.sendArchiveService = new ProduceArchiveService(archiveConfig, context.getClusterManager(), context.getConsume(), context.getMessageConvertSupport()); + this.sendArchiveService = new ProduceArchiveService(archiveConfig, context, rateLimiterManager); } if (consumeArchiveService == null) { - this.consumeArchiveService = new ConsumeArchiveService(archiveConfig, context.getClusterManager()); + this.consumeArchiveService = new ConsumeArchiveService(archiveConfig, context, rateLimiterManager); } } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java index 90e4d7618..0741c4f75 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java @@ -18,11 +18,16 @@ import com.google.common.base.Preconditions; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.io.FileUtils; +import org.joyqueue.broker.BrokerContext; import org.joyqueue.broker.Plugins; import org.joyqueue.broker.cluster.ClusterManager; +import org.joyqueue.broker.limit.RateLimiter; +import org.joyqueue.broker.limit.SubscribeRateLimiter; +import org.joyqueue.domain.Subscription; import org.joyqueue.exception.JoyQueueException; import org.joyqueue.message.MessageLocation; import org.joyqueue.monitor.PointTracer; +import org.joyqueue.monitor.TraceStat; import org.joyqueue.network.session.Connection; import org.joyqueue.server.archive.store.api.ArchiveStore; import org.joyqueue.server.archive.store.model.ConsumeLog; @@ -62,6 +67,7 @@ public class ConsumeArchiveService extends Service { private ClusterManager clusterManager; // 归档日志 private ArchiveConfig archiveConfig; + private BrokerContext brokerContext; // 统计当前读取的字节数,用于异常回滚 private AtomicInteger readByteCounter; @@ -74,11 +80,21 @@ public class ConsumeArchiveService extends Service { private PointTracer tracer; + // 归档限流器 + private SubscribeRateLimiter rateLimiterManager; + public ConsumeArchiveService(ArchiveConfig archiveConfig, ClusterManager clusterManager) { this.clusterManager = clusterManager; this.archiveConfig = archiveConfig; } + public ConsumeArchiveService(ArchiveConfig archiveConfig, BrokerContext brokerContext, SubscribeRateLimiter rateLimiter) { + this.archiveConfig = archiveConfig; + this.brokerContext = brokerContext; + this.clusterManager = brokerContext.getClusterManager(); + this.rateLimiterManager = rateLimiter; + } + @Override protected void validate() throws Exception { super.validate(); @@ -257,13 +273,30 @@ public void appendConsumeLog(Connection connection, MessageLocation[] locations) logger.warn("ConsumeArchiveService not be started."); return; } - List logList = convert(connection, locations); - logList.forEach(log -> { - // 序列化 - ByteBuffer buffer = ArchiveSerializer.write(log); - appendLog(buffer); - ArchiveSerializer.release(buffer); - }); + TraceStat stat = tracer.begin("org.joyqueue.server.archive.consume.appendConsumeLog"); + if (locations != null && locations.length > 0) { + MessageLocation location = locations[0]; + if (checkRateLimit(connection, location)) { + List logList = convert(connection, locations); + logList.forEach(log -> { + // 序列化 + ByteBuffer buffer = ArchiveSerializer.write(log); + appendLog(buffer); + ArchiveSerializer.release(buffer); + }); + } else { + logger.error("Consume-archive: trigger rate limited topic: {}, app: {}", location.getTopic(), connection.getApp()); + } + } + tracer.end(stat); + } + + private boolean checkRateLimit(Connection connection, MessageLocation messageLocation) { + RateLimiter rateLimiter = rateLimiterManager.getOrCreate(messageLocation.getTopic(), connection.getApp(), Subscription.Type.CONSUMPTION); + if(rateLimiter == null || rateLimiter.tryAcquireTps()) { + return true; + } + return false; } /** diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java index 3015848d8..a300757da 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java @@ -17,12 +17,16 @@ import com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; +import org.joyqueue.broker.BrokerContext; import org.joyqueue.broker.Plugins; import org.joyqueue.broker.buffer.Serializer; import org.joyqueue.broker.cluster.ClusterManager; import org.joyqueue.broker.consumer.Consume; import org.joyqueue.broker.consumer.MessageConvertSupport; import org.joyqueue.broker.consumer.model.PullResult; +import org.joyqueue.broker.limit.RateLimiter; +import org.joyqueue.broker.limit.SubscribeRateLimiter; +import org.joyqueue.domain.Subscription; import org.joyqueue.domain.TopicConfig; import org.joyqueue.domain.TopicName; import org.joyqueue.exception.JoyQueueException; @@ -76,6 +80,7 @@ public class ProduceArchiveService extends Service { private int batchNum = 1000; // 集群管理 private ClusterManager clusterManager; + private BrokerContext brokerContext; // 消费管理 private Consume consume; // 发送归档任务池 @@ -108,6 +113,9 @@ public class ProduceArchiveService extends Service { private ArchiveConfig archiveConfig; private MessageConvertSupport messageConvertSupport; + // 归档限流器 + private SubscribeRateLimiter rateLimiterManager; + public ProduceArchiveService(ArchiveConfig archiveConfig, ClusterManager clusterManager, Consume consume, MessageConvertSupport messageConvertSupport) { this.clusterManager = clusterManager; this.consume = consume; @@ -115,6 +123,15 @@ public ProduceArchiveService(ArchiveConfig archiveConfig, ClusterManager cluster this.messageConvertSupport = messageConvertSupport; } + public ProduceArchiveService(ArchiveConfig archiveConfig, BrokerContext brokerContext, SubscribeRateLimiter rateLimiter) { + this.archiveConfig = archiveConfig; + this.brokerContext = brokerContext; + this.clusterManager = brokerContext.getClusterManager(); + this.consume = brokerContext.getConsume(); + this.messageConvertSupport = brokerContext.getMessageConvertSupport(); + this.rateLimiterManager = rateLimiter; + } + @Override protected void validate() throws Exception { super.validate(); @@ -162,8 +179,6 @@ protected void validate() throws Exception { // 队列读取消息,放入归档存储 write2Store(); }).build(); - - } @Override @@ -282,6 +297,14 @@ private void readArchiveMsg() throws Exception { } } + private boolean checkRateLimit(String topic) { + RateLimiter rateLimiter = rateLimiterManager.getOrCreate(topic, clusterManager.getBrokerId() + "", Subscription.Type.PRODUCTION); + if(rateLimiter == null || rateLimiter.tryAcquireTps()) { + return true; + } + return false; + } + /** * @param topic * @param partition From c2d5e4ae0beda92515aa7fb644dee0417d302014 Mon Sep 17 00:00:00 2001 From: majun87 Date: Sun, 20 Dec 2020 17:44:26 +0800 Subject: [PATCH 09/15] =?UTF-8?q?=E7=94=9F=E4=BA=A7=E5=BD=92=E6=A1=A3?= =?UTF-8?q?=E9=81=BF=E5=85=8Draft=E5=88=87=E6=8D=A2=E5=AF=BC=E8=87=B4?= =?UTF-8?q?=E7=9A=84=E4=B8=A2=E6=95=B0=E6=8D=AE=E9=97=AE=E9=A2=98=EF=BC=8C?= =?UTF-8?q?=E4=BF=AE=E6=94=B9=E5=BD=92=E6=A1=A3=E7=9B=B8=E5=85=B3=E5=8F=82?= =?UTF-8?q?=E6=95=B0=E6=8F=90=E5=8D=87=E5=BD=92=E6=A1=A3=E6=80=A7=E8=83=BD?= =?UTF-8?q?=E3=80=82?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../org/joyqueue/broker/archive/ArchiveConfigKey.java | 6 +++--- .../broker/archive/ProduceArchiveService.java | 11 ++++++++--- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java index 095a84cc4..a70c93168 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java @@ -25,9 +25,9 @@ public enum ArchiveConfigKey implements PropertyDef { CONSUME_BATCH_NUM("archive.consume.batch.num", 1000, Type.INT), CONSUME_WRITE_DELAY("archive.consume.write.delay", 1, Type.INT), - PRODUCE_BATCH_NUM("archive.produce.batch.num", 10, Type.INT), - LOG_QUEUE_SIZE("archive.send.log.queue.size", 1000, Type.INT), - WRITE_THREAD_NUM("archive.thread.num", 5, Type.INT), + PRODUCE_BATCH_NUM("archive.produce.batch.num", 50, Type.INT), + LOG_QUEUE_SIZE("archive.send.log.queue.size", 10000, Type.INT), + WRITE_THREAD_NUM("archive.thread.num", 10, Type.INT), ARCHIVE_SWITCH("archive.switch", false, Type.BOOLEAN), ARCHIVE_THREAD_POOL_QUEUE_SIZE("archive.thread.pool.queue.size", 10, Type.INT), ARCHIVE_STORE_NAMESPACE("archive.store.namespace", "joyqueue", Type.STRING), diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java index a300757da..29b78191e 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java @@ -399,6 +399,7 @@ private SendLog convert(BrokerMessage brokerMessage, ByteBuffer buffer) { private void write2Store() throws InterruptedException { int readBatchSize; do { + batchNum = archiveConfig.getProduceBatchNum(); List sendLogs = new ArrayList<>(batchNum); for (int i = 0; i < batchNum; i++) { SendLog sendLog = archiveQueue.poll(); @@ -415,7 +416,7 @@ private void write2Store() throws InterruptedException { // 写入存储 archiveStore.putSendLog(sendLogs, tracer); if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_PRODUCE_PREFIX, clusterManager.getBrokerId().toString())) { - logger.info("Produce-archive: write sendLogs size:{} to archive store.", sendLogs.size()); + logger.info("Produce-archive: write sendLogs size:{} to archive store. sample log: {}", sendLogs.size(), sendLogs.get(0)); } // 写入计数(用于归档位置) writeCounter(sendLogs); @@ -674,8 +675,11 @@ public List getAll() { public void remove(SendArchiveItem item) throws JoyQueueException { // 移除列表 cpList.remove(item); - // clean archive position from store - archiveStore.cleanPosition(item.getTopic(),item.getPartition()); + // only archive disable will clean position store, not include raft node + if (!clusterManager.checkArchiveable(TopicName.parse(item.getTopic()))) { + logger.info("Produce-archive: topic [{}] archive is disable on clean.", item.getTopic()); + archiveStore.cleanPosition(item.getTopic(),item.getPartition()); + } } /** @@ -709,6 +713,7 @@ public void addAndUpdate(List newItemList) throws JoyQueueExcep // fullName consumer.setTopic(item.getTopic()); index = consume.getMaxIndex(consumer,item.getPartition()); + archiveStore.putPosition(new AchivePosition(item.getTopic(), item.getPartition(), index)); logger.info("Produce-archive: new archive item, topic [{}], partition [{}], init from local store max index {}",item.getTopic(),item.getPartition(),item.getReadIndex()); }else{ logger.info("Produce-archive: new archive item, topic [{}], partition [{}], recover from archive position store index {}",item.getTopic(),item.getPartition(),index); From f24d640b14b4c647b7bdc64c26052fcd1e394424 Mon Sep 17 00:00:00 2001 From: majun87 Date: Wed, 23 Dec 2020 12:22:31 +0800 Subject: [PATCH 10/15] =?UTF-8?q?=E4=BC=98=E5=8C=96=E5=BD=92=E6=A1=A3?= =?UTF-8?q?=E7=94=9F=E4=BA=A7=E5=92=8C=E6=B6=88=E8=B4=B9=E9=99=90=E6=B5=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../store/utils/ArchiveSerializer.java | 19 ++++++++++++++++++ .../broker/archive/ConsumeArchiveService.java | 11 +++++----- .../broker/archive/ProduceArchiveService.java | 8 ++++++-- .../broker/consumer/ConsumeConfigKey.java | 4 ++-- .../broker/producer/ProducerConfigKey.java | 4 ++-- .../broker/retry/BrokerRetryManager.java | 1 + .../broker/archive/ArchiveSerializerTest.java | 20 ++++++++++++++++++- 7 files changed, 54 insertions(+), 13 deletions(-) diff --git a/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/utils/ArchiveSerializer.java b/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/utils/ArchiveSerializer.java index 9dd74dac9..a4da2c6dd 100644 --- a/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/utils/ArchiveSerializer.java +++ b/joyqueue-server/joyqueue-archive/joyqueue-archive-api/src/main/java/org/joyqueue/server/archive/store/utils/ArchiveSerializer.java @@ -261,4 +261,23 @@ public static byte[] hexStrToByteArray(String str) { } return byteArray; } + + public static byte[] reverse(byte[] byteArray) { + if (byteArray == null || byteArray.length == 0) { + return byteArray; + } + byte[] reverseArray = new byte[byteArray.length]; + for (int i = 0; i < byteArray.length; i++) { + reverseArray[i] = byteArray[byteArray.length - i - 1]; + } + return reverseArray; + } + + public static byte[] reverse(ByteBuffer buffer) { + return reverse(buffer.array()); + } + + public static String reverse(String reverseStr) { + return new StringBuffer(reverseStr).reverse().toString(); + } } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java index 0741c4f75..5c7f3d6b6 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java @@ -275,8 +275,7 @@ public void appendConsumeLog(Connection connection, MessageLocation[] locations) } TraceStat stat = tracer.begin("org.joyqueue.server.archive.consume.appendConsumeLog"); if (locations != null && locations.length > 0) { - MessageLocation location = locations[0]; - if (checkRateLimit(connection, location)) { + if (checkRateLimitAvailable(connection, locations)) { List logList = convert(connection, locations); logList.forEach(log -> { // 序列化 @@ -285,15 +284,15 @@ public void appendConsumeLog(Connection connection, MessageLocation[] locations) ArchiveSerializer.release(buffer); }); } else { - logger.error("Consume-archive: trigger rate limited topic: {}, app: {}", location.getTopic(), connection.getApp()); + logger.warn("Consume-archive: trigger rate limited topic: {}, app: {}", locations[0].getTopic(), connection.getApp()); } } tracer.end(stat); } - private boolean checkRateLimit(Connection connection, MessageLocation messageLocation) { - RateLimiter rateLimiter = rateLimiterManager.getOrCreate(messageLocation.getTopic(), connection.getApp(), Subscription.Type.CONSUMPTION); - if(rateLimiter == null || rateLimiter.tryAcquireTps()) { + private boolean checkRateLimitAvailable(Connection connection, MessageLocation[] locations) { + RateLimiter rateLimiter = rateLimiterManager.getOrCreate(locations[0].getTopic(), connection.getApp(), Subscription.Type.CONSUMPTION); + if (rateLimiter == null || rateLimiter.tryAcquireTps(locations.length)) { return true; } return false; diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java index 29b78191e..0eb694f68 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java @@ -241,6 +241,10 @@ private void readArchiveMsg() throws Exception { if (isPause(item.getTopic(), item.getPartition())) { continue; } + if (!checkRateLimitAvailable(item.topic)) { + logger.warn("Produce-archive: trigger rate limited topic: {}", item); + continue; + } PullResult pullResult; long readIndex = item.getReadIndex(); try { @@ -297,9 +301,9 @@ private void readArchiveMsg() throws Exception { } } - private boolean checkRateLimit(String topic) { + private boolean checkRateLimitAvailable(String topic) { RateLimiter rateLimiter = rateLimiterManager.getOrCreate(topic, clusterManager.getBrokerId() + "", Subscription.Type.PRODUCTION); - if(rateLimiter == null || rateLimiter.tryAcquireTps()) { + if (rateLimiter == null || rateLimiter.tryAcquireTps(batchNum)) { return true; } return false; diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java index 4c30a63d1..f6a46609d 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java @@ -30,8 +30,8 @@ public enum ConsumeConfigKey implements PropertyDef { RETRY_RATE_PREFIX("retry.rate.", -1, Type.INT), RETRY_FORCE_ACK("retry.ack.force", false, Type.BOOLEAN), RETRY_FORCE_ACK_PREFIX("retry.ack.force.", false, Type.BOOLEAN), - CONSUME_ARCHIVE_RATE("archive.rate.consume", -1, Type.INT), - CONSUME_ARCHIVE_RATE_PREFIX("archive.rate.consume.", -1, Type.INT), + CONSUME_ARCHIVE_RATE("archive.rate.consume", 1000, Type.INT), + CONSUME_ARCHIVE_RATE_PREFIX("archive.rate.consume.", 1000, Type.INT), BROADCAST_INDEX_RESET_ENABLE("consume.broadcast.index.reset.enable", true, Type.BOOLEAN), BROADCAST_INDEX_RESET_INTERVAL("consume.broadcast.index.reset.interval", 1000 * 60 * 5, Type.INT), BROADCAST_INDEX_RESET_TIME("consume.broadcast.index.reset.time", 1000 * 60 * 60 * 24 * 2, Type.INT), diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java index 362d6abc1..b08736e45 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java @@ -28,8 +28,8 @@ public enum ProducerConfigKey implements PropertyDef { TOPIC_QOS_LEVEL_PREFIX("produce.topic.qos.level.", -1, Type.INT), APP_QOS_LEVEL_PREFIX("produce.app.qos.level.", -1, Type.INT), PRINT_METRIC_INTERVAL_MS("print.metric.interval", 0L ,Type.LONG), - PRODUCE_ARCHIVE_RATE("archive.rate.produce", -1, Type.INT), - PRODUCE_ARCHIVE_RATE_PREFIX("archive.rate.produce.", -1, Type.INT), + PRODUCE_ARCHIVE_RATE("archive.rate.produce", 1000, Type.INT), + PRODUCE_ARCHIVE_RATE_PREFIX("archive.rate.produce.", 1000, Type.INT), // businessId长度 PRODUCE_BUSINESSID_LENGTH("produce.businessId.length", 100, PropertyDef.Type.INT), diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java index 32719cc5e..97b9bbfe2 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java @@ -221,6 +221,7 @@ public void addRetry(List retryMessageModelList) throws JoyQu throw e; } }else{ + logger.warn("Broker retry message limited, limit consumers: {}", consumers); throw new JoyQueueException(JoyQueueCode.RETRY_TOKEN_LIMIT); } } diff --git a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ArchiveSerializerTest.java b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ArchiveSerializerTest.java index 3cfb93d18..1658712d5 100644 --- a/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ArchiveSerializerTest.java +++ b/joyqueue-server/joyqueue-broker-core/src/test/java/org/joyqueue/broker/archive/ArchiveSerializerTest.java @@ -15,10 +15,10 @@ */ package org.joyqueue.broker.archive; -import org.joyqueue.broker.archive.ArchiveSerializer; import org.joyqueue.server.archive.store.model.ConsumeLog; import org.joyqueue.toolkit.time.SystemClock; import org.apache.commons.lang.builder.ToStringBuilder; +import org.junit.Assert; import org.junit.Test; import java.nio.ByteBuffer; @@ -62,4 +62,22 @@ public void read() { System.out.println(ToStringBuilder.reflectionToString(log)); } + + @Test + public void reverseStringTest() { + String test = "abc"; + Assert.assertEquals("cba", + org.joyqueue.server.archive.store.utils.ArchiveSerializer.reverse(test)); + } + + @Test + public void reverseBytesTest() { + String test = "abc"; + byte[] bytes = test.getBytes(); + Assert.assertEquals(new String(bytes), + new String(org.joyqueue.server.archive.store.utils.ArchiveSerializer.reverse( + org.joyqueue.server.archive.store.utils.ArchiveSerializer.reverse(bytes) + )) + ); + } } \ No newline at end of file From aefb1e3d3d3710b4527733385e2da9a6c99c974e Mon Sep 17 00:00:00 2001 From: majun87 Date: Wed, 23 Dec 2020 14:43:42 +0800 Subject: [PATCH 11/15] =?UTF-8?q?=E9=99=90=E6=B5=81=E7=BB=9F=E8=AE=A1?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../org/joyqueue/broker/archive/ConsumeArchiveService.java | 4 ++++ .../org/joyqueue/broker/archive/ProduceArchiveService.java | 5 +++++ .../java/org/joyqueue/broker/retry/BrokerRetryManager.java | 2 ++ 3 files changed, 11 insertions(+) diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java index 5c7f3d6b6..f73e5cbc5 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java @@ -284,7 +284,11 @@ public void appendConsumeLog(Connection connection, MessageLocation[] locations) ArchiveSerializer.release(buffer); }); } else { + TraceStat limitBroker = tracer.begin("archive.consume.rate.limited"); + TraceStat limitTopic = tracer.begin(String.format("archive.produce.rate.limited.%s.%s", locations[0].getTopic(), connection.getApp())); logger.warn("Consume-archive: trigger rate limited topic: {}, app: {}", locations[0].getTopic(), connection.getApp()); + tracer.end(limitBroker); + tracer.end(limitTopic); } } tracer.end(stat); diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java index 0eb694f68..5513784e9 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java @@ -33,6 +33,7 @@ import org.joyqueue.message.BrokerMessage; import org.joyqueue.message.SourceType; import org.joyqueue.monitor.PointTracer; +import org.joyqueue.monitor.TraceStat; import org.joyqueue.network.session.Consumer; import org.joyqueue.server.archive.store.api.ArchiveStore; import org.joyqueue.server.archive.store.model.AchivePosition; @@ -242,7 +243,11 @@ private void readArchiveMsg() throws Exception { continue; } if (!checkRateLimitAvailable(item.topic)) { + TraceStat limitBroker = tracer.begin("archive.produce.rate.limited"); + TraceStat limitTopic = tracer.begin(String.format("archive.produce.rate.limited.%s", item.topic)); logger.warn("Produce-archive: trigger rate limited topic: {}", item); + tracer.end(limitBroker); + tracer.end(limitTopic); continue; } PullResult pullResult; diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java index 97b9bbfe2..01bb4efb7 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryManager.java @@ -221,7 +221,9 @@ public void addRetry(List retryMessageModelList) throws JoyQu throw e; } }else{ + TraceStat limit = tracer.begin("BrokerRetryManager.rate.limited"); logger.warn("Broker retry message limited, limit consumers: {}", consumers); + tracer.end(limit); throw new JoyQueueException(JoyQueueCode.RETRY_TOKEN_LIMIT); } } From b365c2fbfccd86982cb808086dfdadf883f1f2cb Mon Sep 17 00:00:00 2001 From: majun87 Date: Fri, 25 Dec 2020 14:52:43 +0800 Subject: [PATCH 12/15] =?UTF-8?q?=E5=BD=92=E6=A1=A3=E9=99=90=E6=B5=81?= =?UTF-8?q?=E8=AE=BE=E7=BD=AE=E8=B0=83=E6=95=B4=E5=88=B0=E4=B8=BB=E9=A2=98?= =?UTF-8?q?=E7=AD=96=E7=95=A5=E9=85=8D=E7=BD=AE=20=E9=87=8D=E6=9E=84?= =?UTF-8?q?=E7=94=9F=E4=BA=A7=E5=92=8C=E6=B6=88=E8=B4=B9=E9=99=90=E6=B5=81?= =?UTF-8?q?=E5=AE=9E=E7=8E=B0?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../main/java/org/joyqueue/domain/Topic.java | 18 ++++ .../broker/archive/ArchiveConfig.java | 33 ++++++ .../broker/archive/ArchiveConfigKey.java | 6 +- .../broker/archive/ArchiveManager.java | 8 ++ .../archive/ArchiveRateLimiterManager.java | 101 ++++++++++++------ .../broker/archive/ConsumeArchiveService.java | 4 +- .../broker/archive/ProduceArchiveService.java | 25 +++-- .../broker/consumer/ConsumeConfig.java | 10 -- .../broker/consumer/ConsumeConfigKey.java | 2 - .../broker/limit/SubscribeRateLimiter.java | 2 + .../broker/limit/config/LimiterConfig.java | 8 ++ .../AbstractSubscribeRateLimiterManager.java | 69 ++++++------ .../broker/producer/ProduceConfig.java | 10 -- .../broker/producer/ProducerConfigKey.java | 2 - .../retry/BrokerRetryRateLimiterManager.java | 17 ++- 15 files changed, 210 insertions(+), 105 deletions(-) diff --git a/joyqueue-common/joyqueue-model/src/main/java/org/joyqueue/domain/Topic.java b/joyqueue-common/joyqueue-model/src/main/java/org/joyqueue/domain/Topic.java index 66965d08c..e6324db89 100644 --- a/joyqueue-common/joyqueue-model/src/main/java/org/joyqueue/domain/Topic.java +++ b/joyqueue-common/joyqueue-model/src/main/java/org/joyqueue/domain/Topic.java @@ -110,6 +110,8 @@ public int hashCode() { public static class TopicPolicy implements Serializable { private Long storeMaxTime; private Boolean storeCleanKeepUnconsumed; + private Integer produceArchiveTps = 1000; + private Integer consumeArchiveTps = 1000; private Map params; public Long getStoreMaxTime() { @@ -128,6 +130,22 @@ public Boolean getStoreCleanKeepUnconsumed() { return storeCleanKeepUnconsumed; } + public Integer getProduceArchiveTps() { + return produceArchiveTps; + } + + public void setProduceArchiveTps(Integer produceArchiveTps) { + this.produceArchiveTps = produceArchiveTps; + } + + public Integer getConsumeArchiveTps() { + return consumeArchiveTps; + } + + public void setConsumeArchiveTps(Integer consumeArchiveTps) { + this.consumeArchiveTps = consumeArchiveTps; + } + public void setParams(Map params) { this.params = params; } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfig.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfig.java index ca12beaa6..52b4d807b 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfig.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfig.java @@ -15,6 +15,7 @@ */ package org.joyqueue.broker.archive; +import org.apache.commons.lang3.StringUtils; import org.joyqueue.config.BrokerConfigKey; import org.joyqueue.toolkit.config.Property; import org.joyqueue.toolkit.config.PropertySupplier; @@ -120,4 +121,36 @@ public boolean isBacklogEnable() { public int getLogRetainDuration() { return PropertySupplier.getValue(propertySupplier, ArchiveConfigKey.ARCHIVE_LOG_RETAIN_DURATION); } + + public int getProduceArchiveRate() { + return propertySupplier.getValue(ArchiveConfigKey.ARCHIVE_PRODUCE_RATE); + } + + public int getConsumeArchiveRate() { + return propertySupplier.getValue(ArchiveConfigKey.ARCHIVE_CONSUME_RATE); + } + + public int getProduceArchiveRate(String topic, String app) { + if (StringUtils.isEmpty(app)) { + return PropertySupplier.getValue(propertySupplier,ArchiveConfigKey.ARCHIVE_PRODUCE_RATE_PREFIX.getName() + String.format("%s",topic), + ArchiveConfigKey.ARCHIVE_PRODUCE_RATE_PREFIX.getType(), ArchiveConfigKey.ARCHIVE_PRODUCE_RATE_PREFIX.getValue() + ); + } else { + return PropertySupplier.getValue(propertySupplier,ArchiveConfigKey.ARCHIVE_PRODUCE_RATE_PREFIX.getName() + String.format("%s.%s",topic,app), + ArchiveConfigKey.ARCHIVE_PRODUCE_RATE_PREFIX.getType(), ArchiveConfigKey.ARCHIVE_PRODUCE_RATE_PREFIX.getValue() + ); + } + } + + public int getConsumeArchiveRate(String topic, String app) { + if (StringUtils.isEmpty(app)) { + return PropertySupplier.getValue(propertySupplier,ArchiveConfigKey.ARCHIVE_CONSUME_RATE_PREFIX.getName() + String.format("%s",topic), + ArchiveConfigKey.ARCHIVE_CONSUME_RATE_PREFIX.getType(), ArchiveConfigKey.ARCHIVE_CONSUME_RATE_PREFIX.getValue() + ); + } else { + return PropertySupplier.getValue(propertySupplier,ArchiveConfigKey.ARCHIVE_CONSUME_RATE_PREFIX.getName() + String.format("%s.%s",topic,app), + ArchiveConfigKey.ARCHIVE_CONSUME_RATE_PREFIX.getType(), ArchiveConfigKey.ARCHIVE_CONSUME_RATE_PREFIX.getValue() + ); + } + } } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java index a70c93168..358eb3543 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java @@ -35,7 +35,11 @@ public enum ArchiveConfigKey implements PropertyDef { ARCHIVE_REAMING_ENABLE("archive.reaming.enable", true, Type.BOOLEAN), ARCHIVE_BACKLOG_ENABLE("archive.backlog.enable", false, Type.BOOLEAN), ARCHIVE_TRACE_LOG("archive.trace.log.", false, Type.BOOLEAN), - ARCHIVE_LOG_RETAIN_DURATION("archive.log.retain.duration", 24, Type.INT) + ARCHIVE_LOG_RETAIN_DURATION("archive.log.retain.duration", 24, Type.INT), + ARCHIVE_PRODUCE_RATE("archive.rate.produce", 1000, Type.INT), + ARCHIVE_PRODUCE_RATE_PREFIX("archive.rate.produce.", 1000, Type.INT), + ARCHIVE_CONSUME_RATE("archive.rate.consume", 1000, Type.INT), + ARCHIVE_CONSUME_RATE_PREFIX("archive.rate.consume.", 1000, Type.INT), ; private String name; diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveManager.java index 03206e5c0..e558c3367 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveManager.java @@ -141,4 +141,12 @@ public Map getSendBacklogNumByTopic() { } return sendArchiveService.getArchivePosition(); } + + public ArchiveConfig getArchiveConfig() { + return archiveConfig; + } + + public void setArchiveConfig(ArchiveConfig archiveConfig) { + this.archiveConfig = archiveConfig; + } } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java index 91537da70..c341957aa 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java @@ -2,14 +2,11 @@ import org.apache.commons.lang3.StringUtils; import org.joyqueue.broker.BrokerContext; -import org.joyqueue.broker.consumer.ConsumeConfig; -import org.joyqueue.broker.consumer.ConsumeConfigKey; +import org.joyqueue.broker.cluster.ClusterManager; import org.joyqueue.broker.limit.RateLimiter; +import org.joyqueue.broker.limit.config.LimiterConfig; import org.joyqueue.broker.limit.support.AbstractSubscribeRateLimiterManager; -import org.joyqueue.broker.producer.ProduceConfig; -import org.joyqueue.broker.producer.ProducerConfigKey; -import org.joyqueue.domain.Config; -import org.joyqueue.domain.Subscription; +import org.joyqueue.domain.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -21,37 +18,70 @@ * @author majun8 */ public class ArchiveRateLimiterManager extends AbstractSubscribeRateLimiterManager { - protected static final Logger LOG = LoggerFactory.getLogger(ArchiveRateLimiterManager.class); + protected static final Logger logger = LoggerFactory.getLogger(ArchiveRateLimiterManager.class); - private ProduceConfig produceConfig; - private ConsumeConfig consumeConfig; + private ClusterManager clusterManager; + private ArchiveConfig archiveConfig; public ArchiveRateLimiterManager(BrokerContext context) { super(context); - this.produceConfig = new ProduceConfig(context != null ? context.getPropertySupplier() : null);; - this.consumeConfig = new ConsumeConfig(context != null ? context.getPropertySupplier() : null);; + this.clusterManager = context.getClusterManager(); + this.archiveConfig = context.getArchiveManager().getArchiveConfig(); } - @Override - public int producerLimitRate(String topic, String app) { - int archiveRate = produceConfig.getArchiveRate(topic, app); + public int defaultProducerLimitRate(String topic, String app) { + int archiveRate = archiveConfig.getProduceArchiveRate(topic, app); if(archiveRate <= 0) { // get broker level retry rate - archiveRate = produceConfig.getArchiveRate(); + archiveRate = archiveConfig.getProduceArchiveRate(); } return archiveRate; } - @Override - public int consumerLimitRate(String topic, String app) { - int archiveRate = consumeConfig.getArchiveRate(topic, app); + public int defaultConsumerLimitRate(String topic, String app) { + int archiveRate = archiveConfig.getConsumeArchiveRate(topic, app); if(archiveRate <= 0) { // get broker level retry rate - archiveRate = consumeConfig.getArchiveRate(); + archiveRate = archiveConfig.getConsumeArchiveRate(); } return archiveRate; } + @Override + public LimiterConfig getLimiterConfig(String topic, String app, Subscription.Type subscribe) { + TopicConfig topicConfig = clusterManager.getTopicConfig(TopicName.parse(topic)); + switch (subscribe) { + case PRODUCTION: + if (topicConfig != null) { + Topic.TopicPolicy policy = topicConfig.getPolicy(); + Integer tps = policy.getProduceArchiveTps(); + if (tps == null) { + tps = defaultProducerLimitRate(topic, app); + } + if (tps <= 0) { + tps = Integer.MAX_VALUE; + } + return new LimiterConfig(tps, -1); + } + break; + case CONSUMPTION: + if (topicConfig != null) { + Topic.TopicPolicy policy = topicConfig.getPolicy(); + Integer tps = policy.getConsumeArchiveTps(); + if (tps == null) { + tps = defaultConsumerLimitRate(topic, app); + } + if (tps <= 0) { + tps = Integer.MAX_VALUE; + } + return new LimiterConfig(tps, -1); + } + break; + } + logger.error("unsupported limit type, topic: {}, app: {}, type: {}", topic, app, subscribe.name()); + return null; + } + @Override public void cleanRateLimiter(Config config) { String configKey = config.getKey(); @@ -59,7 +89,7 @@ public void cleanRateLimiter(Config config) { return; } - if (StringUtils.equals(configKey, ProducerConfigKey.PRODUCE_ARCHIVE_RATE.getName())) { + if (StringUtils.equals(configKey, ArchiveConfigKey.ARCHIVE_PRODUCE_RATE.getName())) { for (Map.Entry> topic : subscribeRateLimiters.entrySet()) { Iterator> subLimiters = topic.getValue().entrySet().iterator(); while (subLimiters.hasNext()) { @@ -70,18 +100,27 @@ public void cleanRateLimiter(Config config) { } } } - } else if (StringUtils.startsWith(configKey, ProducerConfigKey.PRODUCE_ARCHIVE_RATE_PREFIX.getName())) { + } else if (StringUtils.startsWith(configKey, ArchiveConfigKey.ARCHIVE_PRODUCE_RATE_PREFIX.getName())) { String[] keys = StringUtils.split(configKey, "\\."); - if (keys != null && keys.length == 4) { - String topic = keys[2]; - String app = keys[3]; - if (topic != null && app != null) { - cleanRateLimiter(topic, app, Subscription.Type.PRODUCTION); + if (keys != null) { + if (keys.length == 4) { + // topic prefix + String topic = keys[3]; + if (topic != null) { + cleanRateLimiter(topic, null, Subscription.Type.PRODUCTION); + } + } else if (keys.length == 5) { + // topic & app prefix + String topic = keys[3]; + String app = keys[4]; + if (topic != null && app != null) { + cleanRateLimiter(topic, app, Subscription.Type.PRODUCTION); + } } } } - if (StringUtils.equals(configKey, ConsumeConfigKey.CONSUME_ARCHIVE_RATE.getName())) { + if (StringUtils.equals(configKey, ArchiveConfigKey.ARCHIVE_CONSUME_RATE.getName())) { for (Map.Entry> topic : subscribeRateLimiters.entrySet()) { Iterator> subLimiters = topic.getValue().entrySet().iterator(); while (subLimiters.hasNext()) { @@ -92,11 +131,11 @@ public void cleanRateLimiter(Config config) { } } } - } else if (StringUtils.startsWith(configKey, ConsumeConfigKey.CONSUME_ARCHIVE_RATE_PREFIX.getName())) { + } else if (StringUtils.startsWith(configKey, ArchiveConfigKey.ARCHIVE_CONSUME_RATE_PREFIX.getName())) { String[] keys = StringUtils.split(configKey, "\\."); - if (keys != null && keys.length == 4) { - String topic = keys[2]; - String app = keys[3]; + if (keys != null && keys.length == 5) { + String topic = keys[3]; + String app = keys[4]; if (topic != null && app != null) { cleanRateLimiter(topic, app, Subscription.Type.CONSUMPTION); } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java index f73e5cbc5..47807a358 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java @@ -286,7 +286,9 @@ public void appendConsumeLog(Connection connection, MessageLocation[] locations) } else { TraceStat limitBroker = tracer.begin("archive.consume.rate.limited"); TraceStat limitTopic = tracer.begin(String.format("archive.produce.rate.limited.%s.%s", locations[0].getTopic(), connection.getApp())); - logger.warn("Consume-archive: trigger rate limited topic: {}, app: {}", locations[0].getTopic(), connection.getApp()); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { + logger.warn("Consume-archive: trigger rate limited topic: {}, app: {}", locations[0].getTopic(), connection.getApp()); + } tracer.end(limitBroker); tracer.end(limitTopic); } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java index 5513784e9..7d8dcff94 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ProduceArchiveService.java @@ -211,7 +211,7 @@ protected void doStop() { private void updateArchiveItem() throws JoyQueueException { List list = new ArrayList<>(); List topics = clusterManager.getTopics(); - topics.stream().forEach(topicConfig -> { + topics.forEach(topicConfig -> { TopicName name = topicConfig.getName(); // 检查是否开启发送归档 if (clusterManager.checkArchiveable(name)) { @@ -219,7 +219,7 @@ private void updateArchiveItem() throws JoyQueueException { logger.info("Produce-archive: topic [{}] archive is enable.", name.getFullName()); } List partitionSet = clusterManager.getLocalPartitions(topicConfig); - partitionSet.stream().forEach(partition -> { + partitionSet.forEach(partition -> { list.add(new SendArchiveItem(name.getFullName(), partition)); }); } @@ -245,7 +245,9 @@ private void readArchiveMsg() throws Exception { if (!checkRateLimitAvailable(item.topic)) { TraceStat limitBroker = tracer.begin("archive.produce.rate.limited"); TraceStat limitTopic = tracer.begin(String.format("archive.produce.rate.limited.%s", item.topic)); - logger.warn("Produce-archive: trigger rate limited topic: {}", item); + if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_PRODUCE_PREFIX, clusterManager.getBrokerId().toString())) { + logger.warn("Produce-archive: trigger rate limited topic: {}", item); + } tracer.end(limitBroker); tracer.end(limitTopic); continue; @@ -266,6 +268,7 @@ private void readArchiveMsg() throws Exception { logger.error("Produce-archive: repair read journal message position SendArchiveItem info:[{}], currentIndex-min:[{}], exception: {}", item, minIndex, th.getCause()); + archiveStore.putPosition(new AchivePosition(item.topic, item.partition, minIndex)); } if (th.getCause() instanceof PositionOverflowException) { @@ -276,6 +279,7 @@ private void readArchiveMsg() throws Exception { logger.error("Produce-archive: repair read journal message position SendArchiveItem info:[{}], currentIndex-max:[{}], exception: {}", item, maxIndex, th.getCause()); + archiveStore.putPosition(new AchivePosition(item.topic, item.partition, maxIndex)); } // 报错暂停一会 @@ -307,7 +311,7 @@ private void readArchiveMsg() throws Exception { } private boolean checkRateLimitAvailable(String topic) { - RateLimiter rateLimiter = rateLimiterManager.getOrCreate(topic, clusterManager.getBrokerId() + "", Subscription.Type.PRODUCTION); + RateLimiter rateLimiter = rateLimiterManager.getOrCreate(topic, Subscription.Type.PRODUCTION); if (rateLimiter == null || rateLimiter.tryAcquireTps(batchNum)) { return true; } @@ -686,7 +690,7 @@ public void remove(SendArchiveItem item) throws JoyQueueException { cpList.remove(item); // only archive disable will clean position store, not include raft node if (!clusterManager.checkArchiveable(TopicName.parse(item.getTopic()))) { - logger.info("Produce-archive: topic [{}] archive is disable on clean.", item.getTopic()); + logger.info("Produce-archive: topic [{}] archive is disable on clean.", item); archiveStore.cleanPosition(item.getTopic(),item.getPartition()); } } @@ -716,15 +720,18 @@ public void addAndUpdate(List newItemList) throws JoyQueueExcep // 列表中不包含则添加 if (!cpList.contains(item)) { Long index = archiveStore.getPosition(item.topic, item.partition); + Consumer consumer=new Consumer(); + consumer.setTopic(item.getTopic()); if (index == null) { - // 从当前的 max index 开始归档 - Consumer consumer=new Consumer(); - // fullName - consumer.setTopic(item.getTopic()); index = consume.getMaxIndex(consumer,item.getPartition()); archiveStore.putPosition(new AchivePosition(item.getTopic(), item.getPartition(), index)); logger.info("Produce-archive: new archive item, topic [{}], partition [{}], init from local store max index {}",item.getTopic(),item.getPartition(),item.getReadIndex()); }else{ + Long minIndex = consume.getMinIndex(consumer, item.getPartition()); + if (index < minIndex) { + index = minIndex; + archiveStore.putPosition(new AchivePosition(item.getTopic(), item.getPartition(), index)); + } logger.info("Produce-archive: new archive item, topic [{}], partition [{}], recover from archive position store index {}",item.getTopic(),item.getPartition(),index); } item.setReadIndex(index); diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfig.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfig.java index 5fffa7b72..d1999fe51 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfig.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfig.java @@ -88,16 +88,6 @@ public boolean getRetryForceAck(String topic, String app) { ConsumeConfigKey.RETRY_FORCE_ACK_PREFIX.getValue()); } - public int getArchiveRate() { - return propertySupplier.getValue(ConsumeConfigKey.CONSUME_ARCHIVE_RATE); - } - - public int getArchiveRate(String topic, String app) { - return PropertySupplier.getValue(propertySupplier,ConsumeConfigKey.CONSUME_ARCHIVE_RATE_PREFIX.getName() + String.format("%s.%s",topic,app), - ConsumeConfigKey.CONSUME_ARCHIVE_RATE_PREFIX.getType(), ConsumeConfigKey.CONSUME_ARCHIVE_RATE_PREFIX.getValue() - ); - } - public void setConsumePositionPath(String consumePositionPath) { this.consumePositionPath = consumePositionPath; } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java index f6a46609d..1dc3c4a69 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/consumer/ConsumeConfigKey.java @@ -30,8 +30,6 @@ public enum ConsumeConfigKey implements PropertyDef { RETRY_RATE_PREFIX("retry.rate.", -1, Type.INT), RETRY_FORCE_ACK("retry.ack.force", false, Type.BOOLEAN), RETRY_FORCE_ACK_PREFIX("retry.ack.force.", false, Type.BOOLEAN), - CONSUME_ARCHIVE_RATE("archive.rate.consume", 1000, Type.INT), - CONSUME_ARCHIVE_RATE_PREFIX("archive.rate.consume.", 1000, Type.INT), BROADCAST_INDEX_RESET_ENABLE("consume.broadcast.index.reset.enable", true, Type.BOOLEAN), BROADCAST_INDEX_RESET_INTERVAL("consume.broadcast.index.reset.interval", 1000 * 60 * 5, Type.INT), BROADCAST_INDEX_RESET_TIME("consume.broadcast.index.reset.time", 1000 * 60 * 60 * 24 * 2, Type.INT), diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/SubscribeRateLimiter.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/SubscribeRateLimiter.java index 1fc36ff76..61789dcec 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/SubscribeRateLimiter.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/SubscribeRateLimiter.java @@ -15,4 +15,6 @@ public interface SubscribeRateLimiter extends EventListener { * @return null indicate no limit **/ RateLimiter getOrCreate(String topic, String app, Subscription.Type subscribe); + + RateLimiter getOrCreate(String topic, Subscription.Type subscribe); } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/config/LimiterConfig.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/config/LimiterConfig.java index 005c9e3dd..23130f26e 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/config/LimiterConfig.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/config/LimiterConfig.java @@ -50,4 +50,12 @@ public int getTraffic() { public void setTraffic(int traffic) { this.traffic = traffic; } + + @Override + public String toString() { + return "LimiterConfig{" + + "tps=" + tps + + ", traffic=" + traffic + + '}'; + } } \ No newline at end of file diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java index 3b3a0a515..2eb0eba8f 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java @@ -5,6 +5,7 @@ import org.joyqueue.broker.cluster.ClusterManager; import org.joyqueue.broker.limit.RateLimiter; import org.joyqueue.broker.limit.SubscribeRateLimiter; +import org.joyqueue.broker.limit.config.LimiterConfig; import org.joyqueue.domain.Config; import org.joyqueue.domain.Subscription; import org.joyqueue.event.MetaEvent; @@ -20,7 +21,7 @@ * @author majun8 */ public abstract class AbstractSubscribeRateLimiterManager implements SubscribeRateLimiter { - protected static final Logger LOG = LoggerFactory.getLogger(AbstractSubscribeRateLimiterManager.class); + protected static final Logger logger = LoggerFactory.getLogger(AbstractSubscribeRateLimiterManager.class); protected static final String SPLIT = "."; @@ -41,46 +42,36 @@ public RateLimiter getOrCreate(String topic, String app, Subscription.Type subsc topicRateLimiters = old; } } - RateLimiter subscribeRateLimiter = topicRateLimiters.get(subscribe.name() + SPLIT + app); + String subscribeName = app == null ? subscribe.name() + SPLIT : subscribe.name() + SPLIT + app; + RateLimiter subscribeRateLimiter = topicRateLimiters.get(subscribeName); if (subscribeRateLimiter == null) { - switch (subscribe) { - case PRODUCTION: - int pTps = producerLimitRate(topic, app); - if (pTps > 0) { - subscribeRateLimiter = new DefaultRateLimiter(pTps); - RateLimiter oldRateLimiter = topicRateLimiters.putIfAbsent(subscribe.name() + SPLIT + app, subscribeRateLimiter); - if (oldRateLimiter != null) { - subscribeRateLimiter = oldRateLimiter; - } else { - LOG.info("New produce archive rate limiter for {},{},{},{}", topic, app, subscribe.name(), pTps); - } - } - break; - case CONSUMPTION: - int cTps = consumerLimitRate(topic, app); - if (cTps > 0) { - subscribeRateLimiter = new DefaultRateLimiter(cTps); - RateLimiter oldRateLimiter = topicRateLimiters.putIfAbsent(subscribe.name() + SPLIT + app, subscribeRateLimiter); - if (oldRateLimiter != null) { - subscribeRateLimiter = oldRateLimiter; - } else { - LOG.info("New consume archive rate limiter for {},{},{},{}", topic, app, subscribe.name(), cTps); - } - } - break; + LimiterConfig config = getLimiterConfig(topic, app, subscribe); + subscribeRateLimiter = new DefaultRateLimiter(config.getTps()); + RateLimiter oldRateLimiter = topicRateLimiters.putIfAbsent(subscribeName, subscribeRateLimiter); + if (oldRateLimiter != null) { + subscribeRateLimiter = oldRateLimiter; + } else { + logger.info("Archive rate limiter for {},{},{},{}", topic, app, subscribe.name(), config); } - } return subscribeRateLimiter; } - public abstract int producerLimitRate(String topic, String app); + public RateLimiter getOrCreate(String topic, Subscription.Type subscribe) { + return getOrCreate(topic, null, subscribe); + } - public abstract int consumerLimitRate(String topic, String app); + public abstract LimiterConfig getLimiterConfig(String topic, String app, Subscription.Type subscribe); @Override public void onEvent(MetaEvent event) { switch (event.getEventType()) { + case ADD_CONFIG: { + UpdateConfigEvent updateConfigEvent = (UpdateConfigEvent) event; + Config config = updateConfigEvent.getNewConfig(); + cleanRateLimiter(config); + break; + } case UPDATE_CONFIG: { UpdateConfigEvent updateConfigEvent = (UpdateConfigEvent) event; Config config = updateConfigEvent.getNewConfig(); @@ -93,9 +84,17 @@ public void onEvent(MetaEvent event) { cleanRateLimiter(config); break; } + case ADD_TOPIC: + AddTopicEvent addTopicEvent = (AddTopicEvent) event; + cleanRateLimiter(addTopicEvent.getTopic().getName().getFullName(), null, null); + break; + case UPDATE_TOPIC: + UpdateTopicEvent updateTopicEvent = (UpdateTopicEvent) event; + cleanRateLimiter(updateTopicEvent.getNewTopic().getName().getFullName(), null, null); + break; case REMOVE_TOPIC: - RemoveTopicEvent topicEvent = (RemoveTopicEvent) event; - cleanRateLimiter(topicEvent.getTopic().getName().getFullName(), null, null); + RemoveTopicEvent removeTopicEvent = (RemoveTopicEvent) event; + cleanRateLimiter(removeTopicEvent.getTopic().getName().getFullName(), null, null); break; case UPDATE_PRODUCER: UpdateProducerEvent updateProducerEvent = (UpdateProducerEvent) event; @@ -107,8 +106,7 @@ public void onEvent(MetaEvent event) { RemoveProducerEvent removeProducerEvent = (RemoveProducerEvent) event; cleanRateLimiter(removeProducerEvent.getTopic().getFullName(), removeProducerEvent.getProducer().getApp(), - Subscription.Type.PRODUCTION - ); + Subscription.Type.PRODUCTION); break; case UPDATE_CONSUMER: UpdateConsumerEvent updateConsumerEvent = (UpdateConsumerEvent) event; @@ -120,8 +118,7 @@ public void onEvent(MetaEvent event) { RemoveConsumerEvent removeConsumerEvent = (RemoveConsumerEvent) event; cleanRateLimiter(removeConsumerEvent.getTopic().getFullName(), removeConsumerEvent.getConsumer().getApp(), - Subscription.Type.CONSUMPTION - ); + Subscription.Type.CONSUMPTION); break; } } diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProduceConfig.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProduceConfig.java index 079897fb7..ee6a0b272 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProduceConfig.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProduceConfig.java @@ -84,14 +84,4 @@ public int getBodyLength() { public boolean getLogDetail(String app) { return brokerConfig.getLogDetail(app); } - - public int getArchiveRate() { - return propertySupplier.getValue(ProducerConfigKey.PRODUCE_ARCHIVE_RATE); - } - - public int getArchiveRate(String topic, String app) { - return PropertySupplier.getValue(propertySupplier,ProducerConfigKey.PRODUCE_ARCHIVE_RATE_PREFIX.getName() + String.format("%s.%s",topic,app), - ProducerConfigKey.PRODUCE_ARCHIVE_RATE_PREFIX.getType(), ProducerConfigKey.PRODUCE_ARCHIVE_RATE_PREFIX.getValue() - ); - } } \ No newline at end of file diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java index b08736e45..ab888256f 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/producer/ProducerConfigKey.java @@ -28,8 +28,6 @@ public enum ProducerConfigKey implements PropertyDef { TOPIC_QOS_LEVEL_PREFIX("produce.topic.qos.level.", -1, Type.INT), APP_QOS_LEVEL_PREFIX("produce.app.qos.level.", -1, Type.INT), PRINT_METRIC_INTERVAL_MS("print.metric.interval", 0L ,Type.LONG), - PRODUCE_ARCHIVE_RATE("archive.rate.produce", 1000, Type.INT), - PRODUCE_ARCHIVE_RATE_PREFIX("archive.rate.produce.", 1000, Type.INT), // businessId长度 PRODUCE_BUSINESSID_LENGTH("produce.businessId.length", 100, PropertyDef.Type.INT), diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryRateLimiterManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryRateLimiterManager.java index afcba1080..9a1825c13 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryRateLimiterManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/retry/BrokerRetryRateLimiterManager.java @@ -4,6 +4,7 @@ import org.joyqueue.broker.BrokerContext; import org.joyqueue.broker.consumer.ConsumeConfig; import org.joyqueue.broker.consumer.ConsumeConfigKey; +import org.joyqueue.broker.limit.config.LimiterConfig; import org.joyqueue.broker.limit.support.AbstractSubscribeRateLimiterManager; import org.joyqueue.domain.Config; import org.joyqueue.domain.Subscription; @@ -25,12 +26,22 @@ public BrokerRetryRateLimiterManager(BrokerContext context){ } @Override - public int producerLimitRate(String topic, String app) { + public LimiterConfig getLimiterConfig(String topic, String app, Subscription.Type subscribe) { + if (subscribe == Subscription.Type.CONSUMPTION) { + int tps = defaultConsumerLimitRate(topic, app); + if (tps <= 0) { + tps = Integer.MAX_VALUE; + } + return new LimiterConfig(tps, -1); + } + return null; + } + + public int defaultProducerLimitRate(String topic, String app) { return 0; } - @Override - public int consumerLimitRate(String topic, String app) { + public int defaultConsumerLimitRate(String topic, String app) { int retryRate = consumeConfig.getRetryRate(topic, app); if (retryRate <= 0) { // get broker level retry rate From db764634e30a5a5d8df1266be53c43afc11d6778 Mon Sep 17 00:00:00 2001 From: majun87 Date: Tue, 29 Dec 2020 18:30:18 +0800 Subject: [PATCH 13/15] =?UTF-8?q?=E4=BF=AE=E6=94=B9=E5=BD=92=E6=A1=A3?= =?UTF-8?q?=E5=85=83=E6=95=B0=E6=8D=AE=E5=BC=82=E5=B8=B8=E9=97=AE=E9=A2=98?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../broker/archive/ArchiveRateLimiterManager.java | 14 ++++++++++---- .../AbstractSubscribeRateLimiterManager.java | 4 ++-- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java index c341957aa..e2f74b0a9 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java @@ -20,6 +20,8 @@ public class ArchiveRateLimiterManager extends AbstractSubscribeRateLimiterManager { protected static final Logger logger = LoggerFactory.getLogger(ArchiveRateLimiterManager.class); + private static final int DEFAULT_LIMIT_RATE = 1000; + private ClusterManager clusterManager; private ArchiveConfig archiveConfig; @@ -54,8 +56,10 @@ public LimiterConfig getLimiterConfig(String topic, String app, Subscription.Typ case PRODUCTION: if (topicConfig != null) { Topic.TopicPolicy policy = topicConfig.getPolicy(); - Integer tps = policy.getProduceArchiveTps(); - if (tps == null) { + Integer tps = DEFAULT_LIMIT_RATE; + if (policy != null && policy.getProduceArchiveTps() != null) { + tps = policy.getProduceArchiveTps(); + } else { tps = defaultProducerLimitRate(topic, app); } if (tps <= 0) { @@ -67,8 +71,10 @@ public LimiterConfig getLimiterConfig(String topic, String app, Subscription.Typ case CONSUMPTION: if (topicConfig != null) { Topic.TopicPolicy policy = topicConfig.getPolicy(); - Integer tps = policy.getConsumeArchiveTps(); - if (tps == null) { + Integer tps = DEFAULT_LIMIT_RATE; + if (policy != null && policy.getConsumeArchiveTps() != null) { + tps = policy.getConsumeArchiveTps(); + } else { tps = defaultConsumerLimitRate(topic, app); } if (tps <= 0) { diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java index 2eb0eba8f..f7749f265 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/limit/support/AbstractSubscribeRateLimiterManager.java @@ -67,8 +67,8 @@ public RateLimiter getOrCreate(String topic, Subscription.Type subscribe) { public void onEvent(MetaEvent event) { switch (event.getEventType()) { case ADD_CONFIG: { - UpdateConfigEvent updateConfigEvent = (UpdateConfigEvent) event; - Config config = updateConfigEvent.getNewConfig(); + AddConfigEvent addConfigEvent = (AddConfigEvent) event; + Config config = addConfigEvent.getConfig(); cleanRateLimiter(config); break; } From fd5a8ab6de9e3adf2af3830afce3e1a47993f38c Mon Sep 17 00:00:00 2001 From: majun87 Date: Wed, 30 Dec 2020 19:58:44 +0800 Subject: [PATCH 14/15] =?UTF-8?q?=E9=99=90=E6=B5=81key=E9=94=99=E8=AF=AF?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../org/joyqueue/broker/archive/ArchiveConfigKey.java | 8 ++++---- .../broker/archive/ArchiveRateLimiterManager.java | 2 +- .../joyqueue/broker/archive/ConsumeArchiveService.java | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java index 358eb3543..0a05a9799 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveConfigKey.java @@ -36,10 +36,10 @@ public enum ArchiveConfigKey implements PropertyDef { ARCHIVE_BACKLOG_ENABLE("archive.backlog.enable", false, Type.BOOLEAN), ARCHIVE_TRACE_LOG("archive.trace.log.", false, Type.BOOLEAN), ARCHIVE_LOG_RETAIN_DURATION("archive.log.retain.duration", 24, Type.INT), - ARCHIVE_PRODUCE_RATE("archive.rate.produce", 1000, Type.INT), - ARCHIVE_PRODUCE_RATE_PREFIX("archive.rate.produce.", 1000, Type.INT), - ARCHIVE_CONSUME_RATE("archive.rate.consume", 1000, Type.INT), - ARCHIVE_CONSUME_RATE_PREFIX("archive.rate.consume.", 1000, Type.INT), + ARCHIVE_PRODUCE_RATE("archive.rate.produce", -1, Type.INT), + ARCHIVE_PRODUCE_RATE_PREFIX("archive.rate.produce.", -1, Type.INT), + ARCHIVE_CONSUME_RATE("archive.rate.consume", -1, Type.INT), + ARCHIVE_CONSUME_RATE_PREFIX("archive.rate.consume.", -1, Type.INT), ; private String name; diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java index e2f74b0a9..61663e0e2 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ArchiveRateLimiterManager.java @@ -20,7 +20,7 @@ public class ArchiveRateLimiterManager extends AbstractSubscribeRateLimiterManager { protected static final Logger logger = LoggerFactory.getLogger(ArchiveRateLimiterManager.class); - private static final int DEFAULT_LIMIT_RATE = 1000; + private static final int DEFAULT_LIMIT_RATE = -1; private ClusterManager clusterManager; private ArchiveConfig archiveConfig; diff --git a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java index 47807a358..44f7bb076 100644 --- a/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java +++ b/joyqueue-server/joyqueue-broker-core/src/main/java/org/joyqueue/broker/archive/ConsumeArchiveService.java @@ -285,7 +285,7 @@ public void appendConsumeLog(Connection connection, MessageLocation[] locations) }); } else { TraceStat limitBroker = tracer.begin("archive.consume.rate.limited"); - TraceStat limitTopic = tracer.begin(String.format("archive.produce.rate.limited.%s.%s", locations[0].getTopic(), connection.getApp())); + TraceStat limitTopic = tracer.begin(String.format("archive.consume.rate.limited.%s.%s", locations[0].getTopic(), connection.getApp())); if (archiveConfig.getLogDetail(ArchiveConfig.LOG_DETAIL_CONSUME_PREFIX, clusterManager.getBrokerId().toString())) { logger.warn("Consume-archive: trigger rate limited topic: {}, app: {}", locations[0].getTopic(), connection.getApp()); } From 7e16ca5c00d65379aa890f0e46e53afaee325a40 Mon Sep 17 00:00:00 2001 From: majun87 Date: Tue, 5 Jan 2021 12:08:06 +0800 Subject: [PATCH 15/15] =?UTF-8?q?=E5=BC=80=E6=94=BE=E4=B8=BB=E9=A2=98?= =?UTF-8?q?=E5=BD=92=E6=A1=A3=E9=99=90=E6=B5=81?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/java/org/joyqueue/domain/Topic.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/joyqueue-common/joyqueue-model/src/main/java/org/joyqueue/domain/Topic.java b/joyqueue-common/joyqueue-model/src/main/java/org/joyqueue/domain/Topic.java index a4d2f7365..1dae65b8a 100644 --- a/joyqueue-common/joyqueue-model/src/main/java/org/joyqueue/domain/Topic.java +++ b/joyqueue-common/joyqueue-model/src/main/java/org/joyqueue/domain/Topic.java @@ -121,8 +121,8 @@ public String toString() { public static class TopicPolicy implements Serializable { private Long storeMaxTime; private Boolean storeCleanKeepUnconsumed; - private Integer produceArchiveTps = 1000; - private Integer consumeArchiveTps = 1000; + private Integer produceArchiveTps = -1; + private Integer consumeArchiveTps = -1; private Map params; public Long getStoreMaxTime() {