优化单元测试,优化监控指标,增加限流功能
This commit is contained in:
@@ -40,12 +40,12 @@ public class FileChunkCombiner {
|
||||
SingleOutputStreamOperator<FileChunk> parseMessagePackStream = environment
|
||||
.addSource(KafkaConsumer.byteArrayConsumer(configuration))
|
||||
.name("Kafka Source")
|
||||
.map(new ParseMessagePackMapFunction())
|
||||
.map(new ParseMessagePackMapFunction(configuration.get(Configs.MAP_ENABLE_RATE_LIMIT), configuration.get(Configs.MAP_RATE_LIMIT_THRESHOLD)))
|
||||
.name("Map: Parse Message Pack")
|
||||
.filter(new FileChunkFilterFunction(configuration.getLong(Configs.FILE_MAX_SIZE), configuration.getString(Configs.FILTER_EXPRESSION)))
|
||||
.assignTimestampsAndWatermarks(watermarkStrategy);
|
||||
|
||||
OutputTag<FileChunk> delayedChunkOutputTag = new OutputTag<FileChunk>("delayed-chunk") {
|
||||
OutputTag<FileChunk> delayedChunkOutputTag = new OutputTag<>("delayed-chunk") {
|
||||
};
|
||||
|
||||
List<Trigger<Object, TimeWindow>> triggers = new ArrayList<>();
|
||||
@@ -57,7 +57,7 @@ public class FileChunkCombiner {
|
||||
.window(TumblingEventTimeWindows.of(Time.seconds(configuration.get(Configs.COMBINER_WINDOW_TIME))))
|
||||
.trigger(trigger)
|
||||
.sideOutputLateData(delayedChunkOutputTag)
|
||||
.process(new CombineChunkProcessWindowFunction(configuration))
|
||||
.process(new CombineChunkProcessWindowFunction(configuration.get(Configs.FILE_MAX_CHUNK_COUNT)))
|
||||
.name("Window: Combine Chunk")
|
||||
.setParallelism(configuration.get(Configs.COMBINER_WINDOW_PARALLELISM))
|
||||
.disableChaining();
|
||||
|
||||
@@ -43,6 +43,13 @@ public class Configs {
|
||||
.stringType()
|
||||
.noDefaultValue();
|
||||
|
||||
public static final ConfigOption<Boolean> MAP_ENABLE_RATE_LIMIT = ConfigOptions.key("map.enable.rate.limit")
|
||||
.booleanType()
|
||||
.defaultValue(false);
|
||||
public static final ConfigOption<Long> MAP_RATE_LIMIT_THRESHOLD = ConfigOptions.key("map.rate.limit.threshold")
|
||||
.longType()
|
||||
.defaultValue(Long.MAX_VALUE);
|
||||
|
||||
public static final ConfigOption<Integer> COMBINER_WINDOW_PARALLELISM = ConfigOptions.key("combiner.window.parallelism")
|
||||
.intType()
|
||||
.defaultValue(1);
|
||||
|
||||
@@ -1,27 +1,33 @@
|
||||
package com.zdjizhi.function;
|
||||
|
||||
import com.zdjizhi.config.Configs;
|
||||
import cn.hutool.core.util.ArrayUtil;
|
||||
import cn.hutool.log.Log;
|
||||
import cn.hutool.log.LogFactory;
|
||||
import com.zdjizhi.pojo.FileChunk;
|
||||
import com.zdjizhi.utils.PublicUtil;
|
||||
import com.zdjizhi.utils.StringUtil;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.metrics.Counter;
|
||||
import org.apache.flink.metrics.MeterView;
|
||||
import org.apache.flink.metrics.MetricGroup;
|
||||
import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction;
|
||||
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import static com.zdjizhi.utils.PublicConstants.COMBINE_MODE_SEEK;
|
||||
|
||||
public class CombineChunkProcessWindowFunction extends ProcessWindowFunction<FileChunk, FileChunk, String, TimeWindow> {
|
||||
private static final Log LOG = LogFactory.get();
|
||||
|
||||
private transient Counter duplicateChunkCounter;
|
||||
public transient Counter duplicateChunkCounter;
|
||||
public transient Counter combineErrorCounter;
|
||||
public transient Counter seekChunkCounter;
|
||||
public transient Counter appendChunkCounter;
|
||||
private final Configuration configuration;
|
||||
private final int fileMaxChunkCount;
|
||||
|
||||
public CombineChunkProcessWindowFunction(Configuration configuration) {
|
||||
this.configuration = configuration;
|
||||
public CombineChunkProcessWindowFunction(int fileMaxChunkCount) {
|
||||
this.fileMaxChunkCount = fileMaxChunkCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -30,15 +36,126 @@ public class CombineChunkProcessWindowFunction extends ProcessWindowFunction<Fil
|
||||
MetricGroup metricGroup = getRuntimeContext().getMetricGroup();
|
||||
duplicateChunkCounter = metricGroup.counter("duplicateChunkCount");
|
||||
combineErrorCounter = metricGroup.counter("combineErrorCount");
|
||||
seekChunkCounter = metricGroup.counter("seekChunkCount");
|
||||
appendChunkCounter = metricGroup.counter("appendChunkCount");
|
||||
metricGroup.meter("duplicateChunkPerSecond", new MeterView(duplicateChunkCounter));
|
||||
metricGroup.meter("combineErrorPerSecond", new MeterView(combineErrorCounter));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(String string, Context context, Iterable<FileChunk> elements, Collector<FileChunk> out) {
|
||||
List<FileChunk> fileChunks = PublicUtil.combine(elements, configuration.get(Configs.FILE_MAX_CHUNK_COUNT), duplicateChunkCounter, combineErrorCounter, seekChunkCounter, appendChunkCounter);
|
||||
List<FileChunk> fileChunks = combine(elements);
|
||||
for (FileChunk fileChunk : fileChunks) {
|
||||
out.collect(fileChunk);
|
||||
}
|
||||
}
|
||||
|
||||
private List<FileChunk> combine(Iterable<FileChunk> input) {
|
||||
List<FileChunk> combinedFileChunkList = new ArrayList<>();
|
||||
try {
|
||||
List<FileChunk> originalFileChunkList = StreamSupport.stream(input.spliterator(), false).collect(Collectors.toList());
|
||||
List<byte[]> waitingToCombineChunkList = new ArrayList<>();
|
||||
if (COMBINE_MODE_SEEK.equals(originalFileChunkList.get(0).getCombineMode())) {
|
||||
// 按照offset排序
|
||||
originalFileChunkList.sort(Comparator.comparingLong(FileChunk::getOffset));
|
||||
Iterator<FileChunk> originalFileChunkIterator = originalFileChunkList.iterator();
|
||||
if (originalFileChunkIterator.hasNext()) {
|
||||
int duplicateCount = 0;
|
||||
FileChunk currentFileChunk = originalFileChunkIterator.next();
|
||||
int lastChunkFlag = currentFileChunk.getLastChunkFlag();
|
||||
long startOffset = currentFileChunk.getOffset();
|
||||
if (currentFileChunk.getChunk() != null && currentFileChunk.getChunk().length > 0) {
|
||||
waitingToCombineChunkList.add(currentFileChunk.getChunk());
|
||||
}
|
||||
while (originalFileChunkIterator.hasNext()) {
|
||||
long expectedOffset = currentFileChunk.getOffset() + currentFileChunk.getLength();
|
||||
currentFileChunk = originalFileChunkIterator.next();
|
||||
long actualOffset = currentFileChunk.getOffset();
|
||||
if (expectedOffset > actualOffset) {// 期望offset大于当前offset,该块为重复块,跳过该块
|
||||
duplicateCount++;
|
||||
duplicateChunkCounter.inc();
|
||||
} else if (expectedOffset == actualOffset) {// 期望offset等于当前offset,将该块添加到待合并集合中
|
||||
if (currentFileChunk.getLastChunkFlag() == 1) {
|
||||
lastChunkFlag = currentFileChunk.getLastChunkFlag();
|
||||
}
|
||||
if (currentFileChunk.getChunk() != null && currentFileChunk.getChunk().length > 0) {
|
||||
waitingToCombineChunkList.add(currentFileChunk.getChunk());
|
||||
}
|
||||
} else {// 期望offset小于当前offset,说明缺块
|
||||
if (waitingToCombineChunkList.size() > 0) {//将可合并的chunk合并,清空集合
|
||||
combinedFileChunkList.add(combineChunk(waitingToCombineChunkList, currentFileChunk.getUuid(), currentFileChunk.getFileName(), currentFileChunk.getFileType(), startOffset, currentFileChunk.getCombineMode(), lastChunkFlag, originalFileChunkList.get(0).getMeta(), 0, null));
|
||||
waitingToCombineChunkList.clear();
|
||||
} else {
|
||||
if (lastChunkFlag == 1) {
|
||||
combinedFileChunkList.add(currentFileChunk);
|
||||
}
|
||||
}
|
||||
// 将当前块作为第一个块,继续合并
|
||||
startOffset = currentFileChunk.getOffset();// 重置起始offset
|
||||
lastChunkFlag = currentFileChunk.getLastChunkFlag();
|
||||
if (currentFileChunk.getChunk() != null && currentFileChunk.getChunk().length > 0) {
|
||||
waitingToCombineChunkList.add(currentFileChunk.getChunk());
|
||||
}
|
||||
}
|
||||
}
|
||||
if (waitingToCombineChunkList.size() > 0) {
|
||||
combinedFileChunkList.add(combineChunk(waitingToCombineChunkList, currentFileChunk.getUuid(), currentFileChunk.getFileName(), currentFileChunk.getFileType(), startOffset, currentFileChunk.getCombineMode(), lastChunkFlag, originalFileChunkList.get(0).getMeta(), 0, null));
|
||||
} else {
|
||||
if (lastChunkFlag == 1) {
|
||||
combinedFileChunkList.add(currentFileChunk);
|
||||
}
|
||||
}
|
||||
if (duplicateCount > 0) {
|
||||
LOG.error("Duplicate upload chunk, uuid=" + currentFileChunk.getUuid() + ", repetitionCount=" + duplicateCount);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// 按timestamp排序
|
||||
originalFileChunkList.sort(Comparator.comparingLong(FileChunk::getTimestamp));
|
||||
long startTimestamp = originalFileChunkList.get(0).getTimestamp();
|
||||
StringBuilder timestampAndSizes = new StringBuilder();
|
||||
for (FileChunk originalFileChunk : originalFileChunkList) {
|
||||
byte[] chunk = originalFileChunk.getChunk();
|
||||
if (chunk != null && chunk.length > 0) {
|
||||
chunk = originalFileChunk.getChunk();
|
||||
waitingToCombineChunkList.add(chunk);
|
||||
timestampAndSizes.append(originalFileChunk.getTimestamp()).append("-").append(chunk.length).append(";");
|
||||
}
|
||||
if (waitingToCombineChunkList.size() > fileMaxChunkCount) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (waitingToCombineChunkList.size() > 0) {
|
||||
combinedFileChunkList.add(combineChunk(waitingToCombineChunkList, originalFileChunkList.get(0).getUuid(), originalFileChunkList.get(0).getFileName(), originalFileChunkList.get(0).getFileType(), 0, "append", 0, originalFileChunkList.get(0).getMeta(), startTimestamp, timestampAndSizes.toString()));
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Combiner chunk error.", e);
|
||||
combineErrorCounter.inc();
|
||||
}
|
||||
return combinedFileChunkList;
|
||||
}
|
||||
|
||||
private FileChunk combineChunk(List<byte[]> byteList, String uuid, String fileName, String fileType, long offset, String combineMode, int lastChunkFlag, Map<String, Object> metaMap, long startTimestamp, String chunkNumbers) {
|
||||
FileChunk fileChunk = new FileChunk();
|
||||
fileChunk.setChunkCount(byteList.size());
|
||||
byte[][] bytes = new byte[byteList.size()][];
|
||||
byteList.toArray(bytes);
|
||||
byte[] newData = ArrayUtil.addAll(bytes);
|
||||
if (COMBINE_MODE_SEEK.equals(combineMode)) {
|
||||
fileChunk.setOffset(offset);
|
||||
fileChunk.setLastChunkFlag(lastChunkFlag);
|
||||
} else {
|
||||
if (StringUtil.isNotEmpty(chunkNumbers)) {
|
||||
fileChunk.setChunkNumbers(chunkNumbers);
|
||||
}
|
||||
}
|
||||
fileChunk.setTimestamp(startTimestamp);
|
||||
fileChunk.setFileType(fileType);
|
||||
fileChunk.setUuid(uuid);
|
||||
fileChunk.setChunk(newData);
|
||||
fileChunk.setFileName(fileName);
|
||||
fileChunk.setCombineMode(combineMode);
|
||||
fileChunk.setLength(newData.length);
|
||||
fileChunk.setMeta(metaMap);
|
||||
return fileChunk;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -6,12 +6,13 @@ import org.apache.commons.jexl3.*;
|
||||
import org.apache.flink.api.common.functions.RichFilterFunction;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.metrics.Counter;
|
||||
import org.apache.flink.metrics.MeterView;
|
||||
import org.apache.flink.metrics.MetricGroup;
|
||||
|
||||
public class FileChunkFilterFunction extends RichFilterFunction<FileChunk> {
|
||||
private final long maxFileSize;
|
||||
private final String filterExpression;
|
||||
private transient Counter filterChunkCounter;
|
||||
public transient Counter filterChunkCounter;
|
||||
private JexlExpression jexlExpression;
|
||||
private JexlContext jexlContext;
|
||||
|
||||
@@ -25,6 +26,7 @@ public class FileChunkFilterFunction extends RichFilterFunction<FileChunk> {
|
||||
super.open(parameters);
|
||||
MetricGroup metricGroup = getRuntimeContext().getMetricGroup();
|
||||
filterChunkCounter = metricGroup.counter("filterChunkCount");
|
||||
metricGroup.meter("filterChunkPerSecond", new MeterView(filterChunkCounter));
|
||||
JexlEngine jexlEngine = new JexlBuilder().create();
|
||||
jexlExpression = jexlEngine.createExpression(filterExpression);
|
||||
jexlContext = new MapContext();
|
||||
|
||||
@@ -6,17 +6,68 @@ import cn.hutool.log.Log;
|
||||
import cn.hutool.log.LogFactory;
|
||||
import com.zdjizhi.pojo.FileChunk;
|
||||
import org.apache.flink.api.common.functions.RichMapFunction;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.metrics.*;
|
||||
import org.msgpack.core.MessagePack;
|
||||
import org.msgpack.core.MessageUnpacker;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static com.zdjizhi.utils.PublicConstants.COMBINE_MODE_APPEND;
|
||||
|
||||
public class ParseMessagePackMapFunction extends RichMapFunction<byte[], FileChunk> {
|
||||
private static final Log LOG = LogFactory.get();
|
||||
|
||||
public transient Counter parseMessagePackCounter;
|
||||
public transient Counter parseMessagePackErrorCounter;
|
||||
public transient Counter rateLimitDropCounter;
|
||||
private final boolean enableRateLimit;
|
||||
private final long rateLimitThreshold;
|
||||
private long timestamp;
|
||||
private long count;
|
||||
|
||||
public ParseMessagePackMapFunction(boolean enableRateLimit, long rateLimitThreshold) {
|
||||
this.rateLimitThreshold = rateLimitThreshold;
|
||||
this.enableRateLimit = enableRateLimit;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
MetricGroup metricGroup = getRuntimeContext().getMetricGroup();
|
||||
parseMessagePackCounter = metricGroup.counter("parseMessagePackCount");
|
||||
parseMessagePackErrorCounter = metricGroup.counter("parseMessagePackErrorCount");
|
||||
rateLimitDropCounter = metricGroup.counter("rateLimitDropCount");
|
||||
metricGroup.meter("parseMessagePackPerSecond", new MeterView(parseMessagePackCounter));
|
||||
metricGroup.meter("parseMessagePackErrorPerSecond", new MeterView(parseMessagePackErrorCounter));
|
||||
metricGroup.meter("rateLimitDropPerSecond", new MeterView(rateLimitDropCounter));
|
||||
timestamp = System.currentTimeMillis();
|
||||
count = 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileChunk map(byte[] messagePackData) {
|
||||
FileChunk fileChunk = null;
|
||||
if (enableRateLimit) {
|
||||
count++;
|
||||
if (System.currentTimeMillis() - timestamp < 1000 && count <= rateLimitThreshold) {
|
||||
fileChunk = parseMessagePack(messagePackData);
|
||||
} else if (System.currentTimeMillis() - timestamp < 1000 && count > rateLimitThreshold) {
|
||||
rateLimitDropCounter.inc();
|
||||
} else {
|
||||
rateLimitDropCounter.inc();
|
||||
timestamp = System.currentTimeMillis();
|
||||
count = 0;
|
||||
}
|
||||
} else {
|
||||
fileChunk = parseMessagePack(messagePackData);
|
||||
}
|
||||
return fileChunk;
|
||||
}
|
||||
|
||||
private FileChunk parseMessagePack(byte[] messagePackData) {
|
||||
parseMessagePackCounter.inc();
|
||||
FileChunk fileChunk;
|
||||
try {
|
||||
fileChunk = new FileChunk();
|
||||
@@ -66,10 +117,11 @@ public class ParseMessagePackMapFunction extends RichMapFunction<byte[], FileChu
|
||||
break;
|
||||
}
|
||||
}
|
||||
if ("append".equals(fileChunk.getCombineMode())) {
|
||||
if (COMBINE_MODE_APPEND.equals(fileChunk.getCombineMode())) {
|
||||
fileChunk.setLastChunkFlag(0);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
parseMessagePackErrorCounter.inc();
|
||||
LOG.error("Parse messagePack failed.", e);
|
||||
fileChunk = null;
|
||||
}
|
||||
|
||||
@@ -4,31 +4,29 @@ import com.zdjizhi.pojo.FileChunk;
|
||||
import org.apache.flink.api.common.functions.RichMapFunction;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.metrics.Counter;
|
||||
import org.apache.flink.metrics.MeterView;
|
||||
import org.apache.flink.metrics.MetricGroup;
|
||||
|
||||
import static com.zdjizhi.utils.PublicConstants.COMBINE_MODE_SEEK;
|
||||
import static com.zdjizhi.utils.PublicConstants.COMBINE_MODE_APPEND;
|
||||
|
||||
public class SideOutputMapFunction extends RichMapFunction<FileChunk, FileChunk> {
|
||||
|
||||
private transient Counter pcapDelayedChunkCounter;
|
||||
private transient Counter trafficDelayedChunkCounter;
|
||||
private transient Counter delayedChunkCounter;
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
MetricGroup metricGroup = getRuntimeContext().getMetricGroup();
|
||||
pcapDelayedChunkCounter = metricGroup.counter("pcapDelayedChunkCount");
|
||||
trafficDelayedChunkCounter = metricGroup.counter("trafficDelayedChunkCount");
|
||||
delayedChunkCounter = metricGroup.counter("delayedChunkCount");
|
||||
metricGroup.meter("delayedChunkPerSecond", new MeterView(delayedChunkCounter));
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileChunk map(FileChunk fileChunk) {
|
||||
delayedChunkCounter.inc();
|
||||
fileChunk.setChunkCount(1);
|
||||
if (COMBINE_MODE_SEEK.equals(fileChunk.getCombineMode())) {
|
||||
trafficDelayedChunkCounter.inc();
|
||||
} else {
|
||||
if (COMBINE_MODE_APPEND.equals(fileChunk.getCombineMode())) {
|
||||
fileChunk.setChunkNumbers(fileChunk.getTimestamp() + "-" + fileChunk.getChunk().length + ";");
|
||||
pcapDelayedChunkCounter.inc();
|
||||
}
|
||||
return fileChunk;
|
||||
}
|
||||
|
||||
@@ -167,16 +167,19 @@ public class FileChunk implements Serializable {
|
||||
length == fileChunk.length &&
|
||||
lastChunkFlag == fileChunk.lastChunkFlag &&
|
||||
chunkCount == fileChunk.chunkCount &&
|
||||
timestamp == fileChunk.timestamp &&
|
||||
Objects.equals(uuid, fileChunk.uuid) &&
|
||||
Objects.equals(fileName, fileChunk.fileName) &&
|
||||
Objects.equals(fileType, fileChunk.fileType) &&
|
||||
Arrays.equals(chunk, fileChunk.chunk) &&
|
||||
Objects.equals(combineMode, fileChunk.combineMode);
|
||||
Objects.equals(combineMode, fileChunk.combineMode) &&
|
||||
Objects.equals(meta, fileChunk.meta) &&
|
||||
Objects.equals(chunkNumbers, fileChunk.chunkNumbers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = Objects.hash(uuid, fileName, fileType, offset, length, combineMode, lastChunkFlag, chunkCount);
|
||||
int result = Objects.hash(uuid, fileName, fileType, offset, length, combineMode, lastChunkFlag, chunkCount, timestamp, meta, chunkNumbers);
|
||||
result = 31 * result + Arrays.hashCode(chunk);
|
||||
return result;
|
||||
}
|
||||
|
||||
@@ -11,6 +11,7 @@ import com.zdjizhi.utils.PublicConstants;
|
||||
import com.zdjizhi.utils.PublicUtil;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.metrics.Counter;
|
||||
import org.apache.flink.metrics.MeterView;
|
||||
import org.apache.flink.metrics.MetricGroup;
|
||||
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
@@ -61,6 +62,10 @@ public class HBaseSink extends RichSinkFunction<FileChunk> {
|
||||
sendHBaseErrorCounter = metricGroup.counter("sendHBaseErrorCount");
|
||||
sendHBaseFileCounter = metricGroup.counter("sendHBaseFileCount");
|
||||
sendHBaseChunkCounter = metricGroup.counter("sendHBaseChunkCount");
|
||||
metricGroup.meter("sendHBasePerSecond", new MeterView(sendHBaseCounter, 5));
|
||||
metricGroup.meter("sendHBaseErrorPerSecond", new MeterView(sendHBaseErrorCounter));
|
||||
metricGroup.meter("sendHBaseFilePerSecond", new MeterView(sendHBaseFileCounter));
|
||||
metricGroup.meter("sendHBaseChunkPerSecond", new MeterView(sendHBaseChunkCounter));
|
||||
isAsync = configuration.getBoolean(Configs.SINK_ASYNC);
|
||||
if (isAsync) {
|
||||
AsyncHBaseConnection = HBaseConnectionUtil.getInstance(configuration).getAsyncHBaseConnection();
|
||||
|
||||
@@ -11,6 +11,7 @@ import com.zdjizhi.utils.PublicUtil;
|
||||
import org.apache.commons.lang.CharEncoding;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.metrics.Counter;
|
||||
import org.apache.flink.metrics.MeterView;
|
||||
import org.apache.flink.metrics.MetricGroup;
|
||||
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
|
||||
import org.apache.http.HttpResponse;
|
||||
@@ -69,6 +70,10 @@ public class HosSink extends RichSinkFunction<FileChunk> {
|
||||
sendHosErrorCounter = metricGroup.counter("sendHosErrorCount");
|
||||
sendHosFileCounter = metricGroup.counter("sendHosFileCount");
|
||||
sendHosChunkCounter = metricGroup.counter("sendHosChunkCount");
|
||||
metricGroup.meter("sendHosPerSecond", new MeterView(sendHosCounter, 5));
|
||||
metricGroup.meter("sendHosErrorPerSecond", new MeterView(sendHosErrorCounter));
|
||||
metricGroup.meter("sendHosFilePerSecond", new MeterView(sendHosFileCounter));
|
||||
metricGroup.meter("sendHosChunkPerSecond", new MeterView(sendHosChunkCounter));
|
||||
loadBalanceMode = configuration.getInteger(Configs.SINK_HOS_LOAD_BALANCE_MODE);
|
||||
if (loadBalanceMode == 0) {
|
||||
endpoint = configuration.getString(Configs.SINK_HOS_ENDPOINT);
|
||||
|
||||
@@ -1,134 +1,10 @@
|
||||
package com.zdjizhi.utils;
|
||||
|
||||
import cn.hutool.core.util.ArrayUtil;
|
||||
import cn.hutool.crypto.digest.DigestUtil;
|
||||
import cn.hutool.log.Log;
|
||||
import cn.hutool.log.LogFactory;
|
||||
import com.zdjizhi.pojo.FileChunk;
|
||||
import org.apache.flink.metrics.Counter;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import static com.zdjizhi.utils.PublicConstants.COMBINE_MODE_SEEK;
|
||||
|
||||
public class PublicUtil {
|
||||
private static final Log LOG = LogFactory.get();
|
||||
|
||||
public static List<FileChunk> combine(Iterable<FileChunk> input, long keyMaxChunk, Counter duplicateChunkCounter, Counter combineErrorCounter, Counter seekChunkCounter, Counter appendChunkCounter) {
|
||||
List<FileChunk> combinedFileChunkList = new ArrayList<>();
|
||||
try {
|
||||
List<FileChunk> originalFileChunkList = StreamSupport.stream(input.spliterator(), false).collect(Collectors.toList());
|
||||
List<byte[]> waitingToCombineChunkList = new ArrayList<>();
|
||||
if (COMBINE_MODE_SEEK.equals(originalFileChunkList.get(0).getCombineMode())) {
|
||||
seekChunkCounter.inc();
|
||||
// 按照offset排序
|
||||
originalFileChunkList.sort(Comparator.comparingLong(FileChunk::getOffset));
|
||||
Iterator<FileChunk> originalFileChunkIterator = originalFileChunkList.iterator();
|
||||
if (originalFileChunkIterator.hasNext()) {
|
||||
int duplicateCount = 0;
|
||||
FileChunk currentFileChunk = originalFileChunkIterator.next();
|
||||
int lastChunkFlag = currentFileChunk.getLastChunkFlag();
|
||||
long startOffset = currentFileChunk.getOffset();
|
||||
if (currentFileChunk.getChunk() != null && currentFileChunk.getChunk().length > 0) {
|
||||
waitingToCombineChunkList.add(currentFileChunk.getChunk());
|
||||
}
|
||||
while (originalFileChunkIterator.hasNext()) {
|
||||
seekChunkCounter.inc();
|
||||
long expectedOffset = currentFileChunk.getOffset() + currentFileChunk.getLength();
|
||||
currentFileChunk = originalFileChunkIterator.next();
|
||||
long actualOffset = currentFileChunk.getOffset();
|
||||
if (expectedOffset > actualOffset) {// 期望offset大于当前offset,该块为重复块,跳过该块
|
||||
duplicateCount++;
|
||||
} else if (expectedOffset == actualOffset) {// 期望offset等于当前offset,将该块添加到待合并集合中
|
||||
if (currentFileChunk.getLastChunkFlag() == 1) {
|
||||
lastChunkFlag = currentFileChunk.getLastChunkFlag();
|
||||
}
|
||||
if (currentFileChunk.getChunk() != null && currentFileChunk.getChunk().length > 0) {
|
||||
waitingToCombineChunkList.add(currentFileChunk.getChunk());
|
||||
}
|
||||
} else {// 期望offset小于当前offset,说明缺块
|
||||
if (waitingToCombineChunkList.size() > 0) {//将可合并的chunk合并,清空集合
|
||||
combinedFileChunkList.add(combineChunk(waitingToCombineChunkList, currentFileChunk.getUuid(), currentFileChunk.getFileName(), currentFileChunk.getFileType(), startOffset, currentFileChunk.getCombineMode(), lastChunkFlag, originalFileChunkList.get(0).getMeta(), 0, null));
|
||||
waitingToCombineChunkList.clear();
|
||||
} else {
|
||||
if (lastChunkFlag == 1) {
|
||||
combinedFileChunkList.add(currentFileChunk);
|
||||
}
|
||||
}
|
||||
// 将当前块作为第一个块,继续合并
|
||||
startOffset = currentFileChunk.getOffset();// 重置起始offset
|
||||
lastChunkFlag = currentFileChunk.getLastChunkFlag();
|
||||
if (currentFileChunk.getChunk() != null && currentFileChunk.getChunk().length > 0) {
|
||||
waitingToCombineChunkList.add(currentFileChunk.getChunk());
|
||||
}
|
||||
}
|
||||
}
|
||||
if (waitingToCombineChunkList.size() > 0) {
|
||||
combinedFileChunkList.add(combineChunk(waitingToCombineChunkList, currentFileChunk.getUuid(), currentFileChunk.getFileName(), currentFileChunk.getFileType(), startOffset, currentFileChunk.getCombineMode(), lastChunkFlag, originalFileChunkList.get(0).getMeta(), 0, null));
|
||||
} else {
|
||||
if (lastChunkFlag == 1) {
|
||||
combinedFileChunkList.add(currentFileChunk);
|
||||
}
|
||||
}
|
||||
if (duplicateCount > 0) {
|
||||
LOG.error("Duplicate upload chunk, uuid=" + currentFileChunk.getUuid() + ", repetitionCount=" + duplicateCount);
|
||||
duplicateChunkCounter.inc(duplicateCount);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// 按timestamp排序
|
||||
originalFileChunkList.sort(Comparator.comparingLong(FileChunk::getTimestamp));
|
||||
long startTimestamp = originalFileChunkList.get(0).getTimestamp();
|
||||
StringBuilder timestampAndSizes = new StringBuilder();
|
||||
for (FileChunk originalFileChunk : originalFileChunkList) {
|
||||
appendChunkCounter.inc();
|
||||
byte[] chunk = originalFileChunk.getChunk();
|
||||
if (chunk != null && chunk.length > 0) {
|
||||
chunk = originalFileChunk.getChunk();
|
||||
waitingToCombineChunkList.add(chunk);
|
||||
timestampAndSizes.append(originalFileChunk.getTimestamp()).append("-").append(chunk.length).append(";");
|
||||
}
|
||||
if (waitingToCombineChunkList.size() > keyMaxChunk) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (waitingToCombineChunkList.size() > 0) {
|
||||
combinedFileChunkList.add(combineChunk(waitingToCombineChunkList, originalFileChunkList.get(0).getUuid(), originalFileChunkList.get(0).getFileName(), originalFileChunkList.get(0).getFileType(), 0, "append", 0, originalFileChunkList.get(0).getMeta(), startTimestamp, timestampAndSizes.toString()));
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Combiner file error.", e);
|
||||
combineErrorCounter.inc();
|
||||
}
|
||||
return combinedFileChunkList;
|
||||
}
|
||||
|
||||
private static FileChunk combineChunk(List<byte[]> byteList, String uuid, String fileName, String fileType, long offset, String combineMode, int lastChunkFlag, Map<String, Object> metaMap, long startTimestamp, String chunkNumbers) {
|
||||
FileChunk fileChunk = new FileChunk();
|
||||
fileChunk.setChunkCount(byteList.size());
|
||||
byte[][] bytes = new byte[byteList.size()][];
|
||||
byteList.toArray(bytes);
|
||||
byte[] newData = ArrayUtil.addAll(bytes);
|
||||
if (COMBINE_MODE_SEEK.equals(combineMode)) {
|
||||
fileChunk.setOffset(offset);
|
||||
fileChunk.setLastChunkFlag(lastChunkFlag);
|
||||
} else {
|
||||
if (StringUtil.isNotEmpty(chunkNumbers)) {
|
||||
fileChunk.setChunkNumbers(chunkNumbers);
|
||||
}
|
||||
}
|
||||
fileChunk.setTimestamp(startTimestamp);
|
||||
fileChunk.setFileType(fileType);
|
||||
fileChunk.setUuid(uuid);
|
||||
fileChunk.setChunk(newData);
|
||||
fileChunk.setFileName(fileName);
|
||||
fileChunk.setCombineMode(combineMode);
|
||||
fileChunk.setLength(newData.length);
|
||||
fileChunk.setMeta(metaMap);
|
||||
return fileChunk;
|
||||
}
|
||||
|
||||
public static String getUUID() {
|
||||
return UUID.randomUUID().toString().replace("-", "").toLowerCase();
|
||||
|
||||
Reference in New Issue
Block a user