GAL-224 DoS检测支持知识库动态加载
This commit is contained in:
16
pom.xml
16
pom.xml
@@ -13,6 +13,7 @@
|
||||
<hive.version>2.1.1</hive.version>
|
||||
<hadoop.version>2.7.1</hadoop.version>
|
||||
<scala.binary.version>2.11</scala.binary.version>
|
||||
<jsonpath.version>2.4.0</jsonpath.version>
|
||||
</properties>
|
||||
|
||||
<repositories>
|
||||
@@ -122,6 +123,12 @@
|
||||
<version>1.7.21</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.jayway.jsonpath</groupId>
|
||||
<artifactId>json-path</artifactId>
|
||||
<version>${jsonpath.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.flink</groupId>
|
||||
<artifactId>flink-connector-kafka_2.12</artifactId>
|
||||
@@ -156,6 +163,13 @@
|
||||
</exclusions>
|
||||
</dependency>
|
||||
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-hdfs</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
|
||||
<!-- https://mvnrepository.com/artifact/org.apache.hbase/hbase-client -->
|
||||
<dependency>
|
||||
<groupId>org.apache.hbase</groupId>
|
||||
@@ -211,7 +225,7 @@
|
||||
<dependency>
|
||||
<groupId>com.zdjizhi</groupId>
|
||||
<artifactId>galaxy</artifactId>
|
||||
<version>1.1.0</version>
|
||||
<version>1.1.1</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>slf4j-log4j12</artifactId>
|
||||
|
||||
@@ -10,6 +10,11 @@ import org.jasypt.encryption.pbe.StandardPBEStringEncryptor;
|
||||
*/
|
||||
public class CommonConfig {
|
||||
|
||||
/**
|
||||
* 定位库默认分隔符
|
||||
*/
|
||||
public static final String LOCATION_SEPARATOR = ".";
|
||||
|
||||
private static StandardPBEStringEncryptor encryptor = new StandardPBEStringEncryptor();
|
||||
|
||||
static {
|
||||
@@ -80,6 +85,24 @@ public class CommonConfig {
|
||||
|
||||
public static final int SASL_JAAS_CONFIG_FLAG = CommonConfigurations.getIntProperty("sasl.jaas.config.flag");
|
||||
|
||||
public static final String NACOS_SERVER_ADDR = CommonConfigurations.getStringProperty("nacos.server.addr");
|
||||
public static final String NACOS_USERNAME = CommonConfigurations.getStringProperty("nacos.username");
|
||||
public static final String NACOS_PASSWORD = CommonConfigurations.getStringProperty("nacos.password");
|
||||
public static final String NACOS_DATA_ID = CommonConfigurations.getStringProperty("nacos.data.id");
|
||||
public static final String NACOS_GROUP = CommonConfigurations.getStringProperty("nacos.group");
|
||||
public static final int NACOS_READ_TIMEOUT = CommonConfigurations.getIntProperty("nacos.read.timeout");
|
||||
|
||||
public static final String HOS_TOKEN = CommonConfigurations.getStringProperty("hos.token");
|
||||
|
||||
public static final String CLUSTER_OR_SINGLE = CommonConfigurations.getStringProperty("cluster.or.single");
|
||||
|
||||
public static final String HDFS_URI_NS1 = CommonConfigurations.getStringProperty("hdfs.uri.nn1");
|
||||
public static final String HDFS_URI_NS2 = CommonConfigurations.getStringProperty("hdfs.uri.nn2");
|
||||
public static final String HDFS_PATH = CommonConfigurations.getStringProperty("hdfs.path");
|
||||
public static final String HDFS_USER = CommonConfigurations.getStringProperty("hdfs.user");
|
||||
|
||||
public static final String DOWNLOAD_PATH = CommonConfigurations.getStringProperty("download.path");
|
||||
|
||||
public static void main(String[] args) {
|
||||
StandardPBEStringEncryptor encryptor = new StandardPBEStringEncryptor();
|
||||
// 配置加密解密的密码/salt值
|
||||
|
||||
@@ -1,5 +1,26 @@
|
||||
package com.zdjizhi.common;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
public class CustomFile {
|
||||
public class CustomFile implements Serializable {
|
||||
|
||||
String fileName;
|
||||
|
||||
byte[] content;
|
||||
|
||||
public String getFileName() {
|
||||
return fileName;
|
||||
}
|
||||
|
||||
public void setFileName(String fileName) {
|
||||
this.fileName = fileName;
|
||||
}
|
||||
|
||||
public byte[] getContent() {
|
||||
return content;
|
||||
}
|
||||
|
||||
public void setContent(byte[] content) {
|
||||
this.content = content;
|
||||
}
|
||||
}
|
||||
|
||||
91
src/main/java/com/zdjizhi/common/KnowledgeLog.java
Normal file
91
src/main/java/com/zdjizhi/common/KnowledgeLog.java
Normal file
@@ -0,0 +1,91 @@
|
||||
package com.zdjizhi.common;
|
||||
|
||||
public class KnowledgeLog {
|
||||
public String id;
|
||||
public String name;
|
||||
public String path;
|
||||
public Long size;
|
||||
public String format;
|
||||
public String sha256;
|
||||
public String version;
|
||||
public String updateTime;
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public void setId(String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
public String getName() {
|
||||
return name;
|
||||
}
|
||||
|
||||
public void setName(String name) {
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
public String getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
public void setPath(String path) {
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
public Long getSize() {
|
||||
return size;
|
||||
}
|
||||
|
||||
public void setSize(Long size) {
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
public String getFormat() {
|
||||
return format;
|
||||
}
|
||||
|
||||
public void setFormat(String format) {
|
||||
this.format = format;
|
||||
}
|
||||
|
||||
public String getSha256() {
|
||||
return sha256;
|
||||
}
|
||||
|
||||
public void setSha256(String sha256) {
|
||||
this.sha256 = sha256;
|
||||
}
|
||||
|
||||
public String getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
||||
public void setVersion(String version) {
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
public String getUpdateTime() {
|
||||
return updateTime;
|
||||
}
|
||||
|
||||
public void setUpdateTime(String updateTime) {
|
||||
this.updateTime = updateTime;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "KnowledgeLog{" +
|
||||
"id='" + id + '\'' +
|
||||
", name='" + name + '\'' +
|
||||
", path='" + path + '\'' +
|
||||
", size=" + size +
|
||||
", format='" + format + '\'' +
|
||||
", sha256='" + sha256 + '\'' +
|
||||
", version='" + version + '\'' +
|
||||
", updateTime='" + updateTime + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
@@ -1,22 +1,31 @@
|
||||
package com.zdjizhi.etl;
|
||||
|
||||
import com.alibaba.nacos.api.PropertyKeyConst;
|
||||
import com.fasterxml.jackson.databind.JavaType;
|
||||
import com.zdjizhi.common.CommonConfig;
|
||||
import com.zdjizhi.common.CustomFile;
|
||||
import com.zdjizhi.common.DosSketchLog;
|
||||
import com.zdjizhi.function.BroadcastProcessFunc;
|
||||
import com.zdjizhi.source.DosSketchSource;
|
||||
import com.zdjizhi.utils.FlinkEnvironmentUtils;
|
||||
import com.zdjizhi.utils.JsonMapper;
|
||||
import com.zdjizhi.utils.StringUtil;
|
||||
import org.apache.flink.api.common.eventtime.WatermarkStrategy;
|
||||
import org.apache.flink.api.common.functions.FlatMapFunction;
|
||||
import org.apache.flink.api.common.state.MapStateDescriptor;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.api.common.typeinfo.Types;
|
||||
import org.apache.flink.streaming.api.datastream.BroadcastConnectedStream;
|
||||
import org.apache.flink.streaming.api.datastream.BroadcastStream;
|
||||
import org.apache.flink.streaming.api.datastream.DataStreamSource;
|
||||
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
|
||||
import org.apache.flink.streaming.api.functions.co.BroadcastProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.*;
|
||||
|
||||
/**
|
||||
* @author wlh
|
||||
@@ -34,7 +43,28 @@ public class ParseSketchLog {
|
||||
}
|
||||
|
||||
private static SingleOutputStreamOperator<DosSketchLog> flatSketchSource(){
|
||||
return DosSketchSource.createDosSketchSource().flatMap(new FlatSketchLog());
|
||||
|
||||
DataStreamSource<Map<String, byte[]>> broadcastSource=null;
|
||||
Properties nacosProperties = new Properties();
|
||||
|
||||
nacosProperties.put(PropertyKeyConst.SERVER_ADDR,CommonConfig.NACOS_SERVER_ADDR);
|
||||
nacosProperties.setProperty(PropertyKeyConst.USERNAME, CommonConfig.NACOS_USERNAME);
|
||||
nacosProperties.setProperty(PropertyKeyConst.PASSWORD, CommonConfig.NACOS_PASSWORD);
|
||||
|
||||
if ("CLUSTER".equals(CommonConfig.CLUSTER_OR_SINGLE)){
|
||||
broadcastSource = DosSketchSource.broadcastSource(nacosProperties,CommonConfig.HDFS_PATH);
|
||||
}else {
|
||||
broadcastSource= DosSketchSource.singleBroadcastSource(nacosProperties);
|
||||
}
|
||||
|
||||
MapStateDescriptor<String,Map> descriptor =
|
||||
new MapStateDescriptor<>("descriptorTest", Types.STRING, TypeInformation.of(Map.class));
|
||||
|
||||
BroadcastStream<Map<String, byte[]>> broadcast = broadcastSource.broadcast(descriptor);
|
||||
// BroadcastConnectedStream<String, List<CustomFile>> connect = DosSketchSource.createDosSketchSource().connect(broadcast);
|
||||
return DosSketchSource.createDosSketchSource()
|
||||
.connect(broadcast).process(new BroadcastProcessFunc());
|
||||
// .flatMap(new FlatSketchLog());
|
||||
}
|
||||
|
||||
private static WatermarkStrategy<DosSketchLog> createWatermarkStrategy(){
|
||||
|
||||
76
src/main/java/com/zdjizhi/function/BroadcastProcessFunc.java
Normal file
76
src/main/java/com/zdjizhi/function/BroadcastProcessFunc.java
Normal file
@@ -0,0 +1,76 @@
|
||||
package com.zdjizhi.function;
|
||||
|
||||
import com.fasterxml.jackson.databind.JavaType;
|
||||
import com.zdjizhi.common.CustomFile;
|
||||
import com.zdjizhi.common.DosSketchLog;
|
||||
import com.zdjizhi.etl.ParseSketchLog;
|
||||
import com.zdjizhi.utils.IpUtils;
|
||||
import com.zdjizhi.utils.JsonMapper;
|
||||
import com.zdjizhi.utils.StringUtil;
|
||||
import org.apache.flink.api.common.functions.FlatMapFunction;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.functions.co.BroadcastProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class BroadcastProcessFunc extends BroadcastProcessFunction<String, Map<String, byte[]>, DosSketchLog> {
|
||||
private static Logger logger = LoggerFactory.getLogger(ParseSketchLog.class);
|
||||
private static JsonMapper jsonMapperInstance = JsonMapper.getInstance();
|
||||
private static JavaType hashmapJsonType = jsonMapperInstance.createCollectionType(HashMap.class, String.class, Object.class);
|
||||
private static JavaType listType = jsonMapperInstance.createCollectionType(ArrayList.class, HashMap.class);
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
System.out.println("begin init");
|
||||
IpUtils.loadIpLook();
|
||||
System.out.println("init over");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(String value, ReadOnlyContext ctx, Collector<DosSketchLog> out) throws Exception {
|
||||
try {
|
||||
if (StringUtil.isNotBlank(value)){
|
||||
HashMap<String, Object> sketchSource = jsonMapperInstance.fromJson(value, hashmapJsonType);
|
||||
long sketchStartTime = Long.parseLong(sketchSource.get("sketch_start_time").toString());
|
||||
long sketchDuration = Long.parseLong(sketchSource.get("sketch_duration").toString());
|
||||
String attackType = sketchSource.get("attack_type").toString();
|
||||
ArrayList<HashMap<String, Object>> reportIpList = jsonMapperInstance.fromJson(jsonMapperInstance.toJson(sketchSource.get("report_ip_list")), listType);
|
||||
for (HashMap<String, Object> obj : reportIpList) {
|
||||
DosSketchLog dosSketchLog = new DosSketchLog();
|
||||
dosSketchLog.setSketch_start_time(sketchStartTime);
|
||||
dosSketchLog.setSketch_duration(sketchDuration);
|
||||
dosSketchLog.setAttack_type(attackType);
|
||||
String sourceIp = obj.get("source_ip").toString();
|
||||
String destinationIp = obj.get("destination_ip").toString();
|
||||
long sketchSessions = Long.parseLong(obj.get("sketch_sessions").toString());
|
||||
long sketchPackets = Long.parseLong(obj.get("sketch_packets").toString());
|
||||
long sketchBytes = Long.parseLong(obj.get("sketch_bytes").toString());
|
||||
dosSketchLog.setSource_ip(sourceIp);
|
||||
dosSketchLog.setDestination_ip(destinationIp);
|
||||
dosSketchLog.setSketch_sessions(sketchSessions);
|
||||
dosSketchLog.setSketch_packets(sketchPackets);
|
||||
dosSketchLog.setSketch_bytes(sketchBytes);
|
||||
out.collect(dosSketchLog);
|
||||
logger.debug("数据解析成功:{}",dosSketchLog.toString());
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.error("数据解析错误:{} \n{}",value,e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processBroadcastElement(Map<String, byte[]> value, Context ctx, Collector<DosSketchLog> out) throws Exception {
|
||||
IpUtils.updateIpLook(value);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
@@ -1,12 +1,15 @@
|
||||
package com.zdjizhi.source;
|
||||
|
||||
import com.zdjizhi.common.CommonConfig;
|
||||
import com.zdjizhi.common.CustomFile;
|
||||
import com.zdjizhi.utils.FlinkEnvironmentUtils;
|
||||
import org.apache.flink.api.common.serialization.SimpleStringSchema;
|
||||
import org.apache.flink.streaming.api.datastream.DataStreamSource;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
@@ -31,4 +34,13 @@ public class DosSketchSource {
|
||||
new SimpleStringSchema(), properties))
|
||||
.setParallelism(CommonConfig.KAFKA_INPUT_PARALLELISM);
|
||||
}
|
||||
|
||||
|
||||
public static DataStreamSource<Map<String, byte[]>> broadcastSource(Properties nacosProperties, String STORE_PATH){
|
||||
return streamExeEnv.addSource(new HttpSource(nacosProperties, CommonConfig.NACOS_DATA_ID, CommonConfig.NACOS_GROUP, CommonConfig.NACOS_READ_TIMEOUT,STORE_PATH));
|
||||
}
|
||||
|
||||
public static DataStreamSource<Map<String, byte[]>> singleBroadcastSource(Properties nacosProperties){
|
||||
return streamExeEnv.addSource(new SingleHttpSource(nacosProperties, CommonConfig.NACOS_DATA_ID, CommonConfig.NACOS_GROUP, CommonConfig.NACOS_READ_TIMEOUT));
|
||||
}
|
||||
}
|
||||
|
||||
182
src/main/java/com/zdjizhi/source/HttpSource.java
Normal file
182
src/main/java/com/zdjizhi/source/HttpSource.java
Normal file
@@ -0,0 +1,182 @@
|
||||
package com.zdjizhi.source;
|
||||
|
||||
import cn.hutool.core.io.FileUtil;
|
||||
import cn.hutool.core.io.IoUtil;
|
||||
import cn.hutool.json.JSONObject;
|
||||
import com.alibaba.nacos.api.NacosFactory;
|
||||
import com.alibaba.nacos.api.PropertyKeyConst;
|
||||
import com.alibaba.nacos.api.config.ConfigService;
|
||||
import com.alibaba.nacos.api.config.listener.Listener;
|
||||
import com.fasterxml.jackson.databind.JavaType;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
import com.zdjizhi.common.CommonConfig;
|
||||
import com.zdjizhi.common.CustomFile;
|
||||
import com.zdjizhi.common.KnowledgeLog;
|
||||
import com.zdjizhi.utils.*;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.http.Header;
|
||||
import org.apache.http.message.BasicHeader;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.*;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
|
||||
public class HttpSource extends RichHttpSourceFunction<Map<String, byte[]>> {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(HttpSource.class);
|
||||
|
||||
private static final String EXPR = "$.[?(@.version=='latest' && @.name in ['ip_v4_built_in','ip_v6_built_in','ip_v4_user_defined','ip_v6_user_defined'])].['name','sha256','format','path']";
|
||||
|
||||
//连接nacos的配置
|
||||
private Properties nacosProperties;
|
||||
|
||||
//nacos data id
|
||||
private String NACOS_DATA_ID;
|
||||
|
||||
//nacos group
|
||||
private String NACOS_GROUP;
|
||||
|
||||
//nacos 连接超时时间
|
||||
private long NACOS_READ_TIMEOUT;
|
||||
|
||||
//上传到hdfs的路径
|
||||
private String STORE_PATH;
|
||||
|
||||
private ConfigService configService;
|
||||
|
||||
// private static JsonMapper jsonMapperInstance = JsonMapper.getInstance();
|
||||
// private static JavaType listType = jsonMapperInstance.createCollectionType(List.class, KnowledgeLog.class);
|
||||
private static Map<String, String> updateMap = new HashMap<>();
|
||||
private static HashMap<String, byte[]> knowledgeFileCache;
|
||||
private boolean isRunning = true;
|
||||
|
||||
|
||||
public HttpSource(Properties nacosProperties, String NACOS_DATA_ID, String NACOS_GROUP, long NACOS_READ_TIMEOUT, String storePath) {
|
||||
this.nacosProperties = nacosProperties;
|
||||
this.NACOS_DATA_ID = NACOS_DATA_ID;
|
||||
this.NACOS_GROUP = NACOS_GROUP;
|
||||
this.NACOS_READ_TIMEOUT = NACOS_READ_TIMEOUT;
|
||||
this.STORE_PATH = storePath;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
//初始化元数据缓存
|
||||
updateMap = new HashMap<>(16);
|
||||
//初始化定位库缓存
|
||||
knowledgeFileCache = new HashMap<>(16);
|
||||
logger.info("连接nacos:" + nacosProperties.getProperty(PropertyKeyConst.SERVER_ADDR));
|
||||
configService = NacosFactory.createConfigService(nacosProperties);
|
||||
}
|
||||
@Override
|
||||
public void run(SourceContext ctx) throws Exception {
|
||||
// ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
|
||||
String config = configService.getConfig(NACOS_DATA_ID, NACOS_GROUP, NACOS_READ_TIMEOUT);
|
||||
SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
|
||||
String format = formatter.format(new Date());
|
||||
logger.info(format + "receive config from nacos:" + config);
|
||||
System.out.println(format + "receive config from nacos:" + config);
|
||||
if (StringUtil.isNotBlank(config)) {
|
||||
ArrayList<Object> metaList = JsonPath.parse(config).read(EXPR);
|
||||
loadKnowledge(metaList);
|
||||
}
|
||||
|
||||
|
||||
configService.addListener(NACOS_DATA_ID, NACOS_GROUP, new Listener() {
|
||||
@Override
|
||||
public Executor getExecutor() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receiveConfigInfo(String configMsg) {
|
||||
try {
|
||||
logger.info("receive update config:" + configMsg);
|
||||
if (StringUtil.isNotBlank(configMsg)) {
|
||||
ArrayList<Object> metaList = JsonPath.parse(configMsg).read(EXPR);
|
||||
if (metaList.size() >= 1) {
|
||||
for (Object metadata : metaList) {
|
||||
JSONObject knowledgeJson = new JSONObject(metadata, false, true);
|
||||
String fileName = Joiner.on(CommonConfig.LOCATION_SEPARATOR).useForNull("").join(knowledgeJson.getStr("name"),
|
||||
knowledgeJson.getStr("format"));
|
||||
String sha256 = knowledgeJson.getStr("sha256");
|
||||
String filePath = knowledgeJson.getStr("path");
|
||||
if (!sha256.equals(updateMap.get(fileName))) {
|
||||
updateMap.put(fileName, sha256);
|
||||
updateKnowledge(fileName, filePath);
|
||||
}
|
||||
|
||||
}
|
||||
ctx.collect(knowledgeFileCache);
|
||||
}
|
||||
}
|
||||
|
||||
} catch (Exception e) {
|
||||
logger.error("监听nacos配置失败", e);
|
||||
}
|
||||
System.out.println(configMsg);
|
||||
}
|
||||
});
|
||||
|
||||
while (isRunning) {
|
||||
Thread.sleep(10000);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void loadKnowledge(ArrayList<Object> metaList) {
|
||||
InputStream inputStream = null;
|
||||
try {
|
||||
if (metaList.size() >= 1) {
|
||||
for (Object metadata : metaList) {
|
||||
JSONObject knowledgeJson = new JSONObject(metadata, false, true);
|
||||
String fileName = Joiner.on(CommonConfig.LOCATION_SEPARATOR).useForNull("").join(knowledgeJson.getStr("name"),
|
||||
knowledgeJson.getStr("format"));
|
||||
String sha256 = knowledgeJson.getStr("sha256");
|
||||
String filePath = knowledgeJson.getStr("path");
|
||||
Header header = new BasicHeader("token", CommonConfig.HOS_TOKEN);
|
||||
HttpClientUtils2 httpClientUtils = new HttpClientUtils2();
|
||||
inputStream = httpClientUtils.httpGetInputStream(filePath, 3000, header);
|
||||
updateMap.put(fileName, sha256);
|
||||
knowledgeFileCache.put(fileName, IOUtils.toByteArray(inputStream));
|
||||
}
|
||||
}
|
||||
} catch (IOException ioException) {
|
||||
ioException.printStackTrace();
|
||||
} finally {
|
||||
IOUtils.closeQuietly(inputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void updateKnowledge(String fileName, String filePath) {
|
||||
InputStream inputStream = null;
|
||||
FileOutputStream outputStream = null;
|
||||
try {
|
||||
Header header = new BasicHeader("token", CommonConfig.HOS_TOKEN);
|
||||
HttpClientUtils2 httpClientUtils = new HttpClientUtils2();
|
||||
inputStream = httpClientUtils.httpGetInputStream(filePath, 3000, header);
|
||||
byte[] bytes = IOUtils.toByteArray(inputStream);
|
||||
HdfsUtils.uploadFileByBytes(CommonConfig.HDFS_PATH + fileName, bytes);
|
||||
knowledgeFileCache.put(fileName, bytes);
|
||||
} catch (IOException ioException) {
|
||||
ioException.printStackTrace();
|
||||
} finally {
|
||||
IOUtils.closeQuietly(inputStream);
|
||||
IOUtils.closeQuietly(outputStream);
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public void cancel() {
|
||||
this.isRunning = false;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
6
src/main/java/com/zdjizhi/source/HttpSourceFunction.java
Normal file
6
src/main/java/com/zdjizhi/source/HttpSourceFunction.java
Normal file
@@ -0,0 +1,6 @@
|
||||
package com.zdjizhi.source;
|
||||
|
||||
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||
|
||||
public interface HttpSourceFunction<OUT> extends SourceFunction<OUT> {
|
||||
}
|
||||
10
src/main/java/com/zdjizhi/source/RichHttpSourceFunction.java
Normal file
10
src/main/java/com/zdjizhi/source/RichHttpSourceFunction.java
Normal file
@@ -0,0 +1,10 @@
|
||||
package com.zdjizhi.source;
|
||||
|
||||
import org.apache.flink.api.common.functions.AbstractRichFunction;
|
||||
|
||||
public abstract class RichHttpSourceFunction<OUT> extends AbstractRichFunction implements HttpSourceFunction<OUT> {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public RichHttpSourceFunction() {
|
||||
}
|
||||
}
|
||||
237
src/main/java/com/zdjizhi/source/SingleHttpSource.java
Normal file
237
src/main/java/com/zdjizhi/source/SingleHttpSource.java
Normal file
@@ -0,0 +1,237 @@
|
||||
package com.zdjizhi.source;
|
||||
|
||||
import cn.hutool.core.io.FileUtil;
|
||||
import cn.hutool.core.io.IoUtil;
|
||||
import cn.hutool.json.JSONObject;
|
||||
import com.alibaba.nacos.api.NacosFactory;
|
||||
import com.alibaba.nacos.api.PropertyKeyConst;
|
||||
import com.alibaba.nacos.api.config.ConfigService;
|
||||
import com.alibaba.nacos.api.config.listener.Listener;
|
||||
import com.fasterxml.jackson.databind.JavaType;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
import com.zdjizhi.common.CommonConfig;
|
||||
import com.zdjizhi.common.CustomFile;
|
||||
import com.zdjizhi.common.KnowledgeLog;
|
||||
import com.zdjizhi.utils.*;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.watermark.Watermark;
|
||||
import org.apache.http.Header;
|
||||
import org.apache.http.message.BasicHeader;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
public class SingleHttpSource extends RichHttpSourceFunction<Map<String, byte[]>> {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(HttpSource.class);
|
||||
private static HashMap<String, byte[]> knowledgeFileCache;
|
||||
|
||||
private Properties nacosProperties;
|
||||
|
||||
private String NACOS_DATA_ID;
|
||||
|
||||
private String NACOS_GROUP;
|
||||
|
||||
private long NACOS_READ_TIMEOUT;
|
||||
|
||||
private static String STORE_PATH;
|
||||
|
||||
private ConfigService configService;
|
||||
|
||||
// private static JsonMapper jsonMapperInstance = JsonMapper.getInstance();
|
||||
// private static JavaType listType = jsonMapperInstance.createCollectionType(List.class, KnowledgeLog.class);
|
||||
private static final String EXPR = "$.[?(@.version=='latest' && @.name in ['ip_v4_built_in','ip_v6_built_in','ip_v4_user_defined','ip_v6_user_defined'])].['name','sha256','format','path']";
|
||||
|
||||
|
||||
private static Map<String, String> updateMap = new HashMap<>();
|
||||
|
||||
private boolean isRunning = true;
|
||||
|
||||
|
||||
public SingleHttpSource(Properties nacosProperties, String NACOS_DATA_ID, String NACOS_GROUP, long NACOS_READ_TIMEOUT) {
|
||||
this.nacosProperties = nacosProperties;
|
||||
this.NACOS_DATA_ID = NACOS_DATA_ID;
|
||||
this.NACOS_GROUP = NACOS_GROUP;
|
||||
this.NACOS_READ_TIMEOUT = NACOS_READ_TIMEOUT;
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
logger.info("连接nacos:" + nacosProperties.getProperty(PropertyKeyConst.SERVER_ADDR));
|
||||
configService = NacosFactory.createConfigService(nacosProperties);
|
||||
//初始化元数据缓存
|
||||
updateMap = new HashMap<>(16);
|
||||
//初始化定位库缓存
|
||||
knowledgeFileCache = new HashMap<>(16);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(SourceContext ctx) throws Exception {
|
||||
// ctx.emitWatermark(new Watermark(Long.MAX_VALUE));
|
||||
String config = configService.getConfig(NACOS_DATA_ID, NACOS_GROUP, NACOS_READ_TIMEOUT);
|
||||
// List<CustomFile> customFiles = new ArrayList<>();
|
||||
if (StringUtil.isNotBlank(config)) {
|
||||
ArrayList<Object> metaList = JsonPath.parse(config).read(EXPR);
|
||||
loadKnowledge(metaList);
|
||||
}
|
||||
// if (StringUtil.isNotBlank(config)) {
|
||||
// List<KnowledgeLog> knowledgeLogListList = jsonMapperInstance.fromJson(config, listType);
|
||||
// if (knowledgeLogListList.size()>=1){
|
||||
// for (KnowledgeLog knowledgeLog : knowledgeLogListList) {
|
||||
// String name = knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat());
|
||||
// String sha256 = knowledgeLog.getSha256();
|
||||
// updateMap.put(name,sha256);
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
|
||||
configService.addListener(NACOS_DATA_ID, NACOS_GROUP, new Listener() {
|
||||
@Override
|
||||
public Executor getExecutor() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receiveConfigInfo(String configMsg) {
|
||||
try {
|
||||
logger.info("receive update config:" + configMsg);
|
||||
if (StringUtil.isNotBlank(configMsg)) {
|
||||
ArrayList<Object> metaList = JsonPath.parse(configMsg).read(EXPR);
|
||||
if (metaList.size() >= 1) {
|
||||
for (Object metadata : metaList) {
|
||||
JSONObject knowledgeJson = new JSONObject(metadata, false, true);
|
||||
String fileName = Joiner.on(CommonConfig.LOCATION_SEPARATOR).useForNull("").join(knowledgeJson.getStr("name"),
|
||||
knowledgeJson.getStr("format"));
|
||||
String sha256 = knowledgeJson.getStr("sha256");
|
||||
String filePath = knowledgeJson.getStr("path");
|
||||
if (!sha256.equals(updateMap.get(fileName))) {
|
||||
updateMap.put(fileName, sha256);
|
||||
updateKnowledge(fileName, filePath);
|
||||
}
|
||||
|
||||
}
|
||||
ctx.collect(knowledgeFileCache);
|
||||
}
|
||||
}
|
||||
// ArrayList<CustomFile> customFiles = new ArrayList<>();
|
||||
// List<KnowledgeLog> knowledgeInfo = jsonMapperInstance.fromJson(configMsg, listType);
|
||||
// for (KnowledgeLog knowledgeLog : knowledgeInfo) {
|
||||
// String fileName = knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat());
|
||||
// String sha256 = knowledgeLog.getSha256();
|
||||
// String filePath = knowledgeLog.getPath();
|
||||
// String version = knowledgeLog.getVersion();
|
||||
// if (updateMap.containsKey(fileName)){
|
||||
// if ((!sha256.equals(updateMap.get(fileName)))&& ("latest".equals(version))){
|
||||
// CustomFile customFile = loadKnowledge(fileName, filePath);
|
||||
// customFiles.add(customFile);
|
||||
// updateMap.put(fileName,sha256);
|
||||
// }
|
||||
// }else {
|
||||
// updateMap.put(fileName, sha256);
|
||||
// CustomFile customFile = loadKnowledge(fileName, filePath);
|
||||
// customFiles.add(customFile);
|
||||
// }
|
||||
// }
|
||||
//// customFiles=loadKnowledge(configMsg);
|
||||
//// ctx.collectWithTimestamp(customFiles,System.currentTimeMillis());
|
||||
// ctx.collect(customFiles);
|
||||
|
||||
} catch (Exception e) {
|
||||
logger.error("监听nacos配置失败", e);
|
||||
}
|
||||
System.out.println(configMsg);
|
||||
}
|
||||
});
|
||||
|
||||
while (isRunning) {
|
||||
Thread.sleep(10000);
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
// private CustomFile loadKnowledge(String fileName, String filePath) {
|
||||
// InputStream inputStream = null;
|
||||
// FileOutputStream outputStream = null;
|
||||
// CustomFile customFile = new CustomFile();
|
||||
// try {
|
||||
// customFile.setFileName(fileName);
|
||||
// Header header = new BasicHeader("token", CommonConfig.HOS_TOKEN);
|
||||
// HttpClientUtils2 httpClientUtils = new HttpClientUtils2();
|
||||
// inputStream = httpClientUtils.httpGetInputStream(filePath, 3000, header);
|
||||
// FileUtil.mkdir(CommonConfig.DOWNLOAD_PATH);
|
||||
// File file = new File(CommonConfig.DOWNLOAD_PATH.concat(File.separator).concat(fileName));
|
||||
// outputStream = new FileOutputStream(file);
|
||||
// byte[] bytes = IOUtils.toByteArray(inputStream);
|
||||
// customFile.setContent(bytes);
|
||||
// inputStream = new ByteArrayInputStream(customFile.getContent());
|
||||
// IoUtil.copy(inputStream, outputStream);
|
||||
//
|
||||
// } catch (IOException ioException) {
|
||||
// ioException.printStackTrace();
|
||||
// } finally {
|
||||
// IOUtils.closeQuietly(inputStream);
|
||||
// IOUtils.closeQuietly(outputStream);
|
||||
// }
|
||||
// return customFile;
|
||||
// }
|
||||
private void loadKnowledge(ArrayList<Object> metaList) {
|
||||
InputStream inputStream = null;
|
||||
try {
|
||||
if (metaList.size() >= 1) {
|
||||
for (Object metadata : metaList) {
|
||||
JSONObject knowledgeJson = new JSONObject(metadata, false, true);
|
||||
String fileName = Joiner.on(CommonConfig.LOCATION_SEPARATOR).useForNull("").join(knowledgeJson.getStr("name"),
|
||||
knowledgeJson.getStr("format"));
|
||||
String sha256 = knowledgeJson.getStr("sha256");
|
||||
String filePath = knowledgeJson.getStr("path");
|
||||
Header header = new BasicHeader("token", CommonConfig.HOS_TOKEN);
|
||||
HttpClientUtils2 httpClientUtils = new HttpClientUtils2();
|
||||
inputStream = httpClientUtils.httpGetInputStream(filePath, 3000, header);
|
||||
updateMap.put(fileName, sha256);
|
||||
knowledgeFileCache.put(fileName, IOUtils.toByteArray(inputStream));
|
||||
}
|
||||
}
|
||||
} catch (IOException ioException) {
|
||||
ioException.printStackTrace();
|
||||
} finally {
|
||||
IOUtils.closeQuietly(inputStream);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void updateKnowledge(String fileName, String filePath) {
|
||||
InputStream inputStream = null;
|
||||
FileOutputStream outputStream = null;
|
||||
try {
|
||||
Header header = new BasicHeader("token", CommonConfig.HOS_TOKEN);
|
||||
HttpClientUtils2 httpClientUtils = new HttpClientUtils2();
|
||||
inputStream = httpClientUtils.httpGetInputStream(filePath, 3000, header);
|
||||
FileUtil.mkdir(CommonConfig.DOWNLOAD_PATH);
|
||||
File file = new File(CommonConfig.DOWNLOAD_PATH.concat(File.separator).concat(fileName));
|
||||
outputStream = new FileOutputStream(file);
|
||||
byte[] bytes = IOUtils.toByteArray(inputStream);
|
||||
knowledgeFileCache.put(fileName, bytes);
|
||||
} catch (IOException ioException) {
|
||||
ioException.printStackTrace();
|
||||
} finally {
|
||||
IOUtils.closeQuietly(inputStream);
|
||||
IOUtils.closeQuietly(outputStream);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cancel() {
|
||||
this.isRunning = false;
|
||||
}
|
||||
}
|
||||
|
||||
24
src/main/java/com/zdjizhi/utils/FileByteUtils.java
Normal file
24
src/main/java/com/zdjizhi/utils/FileByteUtils.java
Normal file
@@ -0,0 +1,24 @@
|
||||
package com.zdjizhi.utils;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
|
||||
public class FileByteUtils {
|
||||
|
||||
public static byte[] getFileBytes (String filePath) throws IOException {
|
||||
File file = new File(filePath);
|
||||
FileInputStream fis = new FileInputStream(file);
|
||||
ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
|
||||
byte[] b = new byte[1024];
|
||||
int n;
|
||||
while ((n = fis.read(b)) != -1) {
|
||||
bos.write(b, 0, n);
|
||||
}
|
||||
fis.close();
|
||||
byte[] data = bos.toByteArray();
|
||||
bos.close();
|
||||
return data;
|
||||
}
|
||||
}
|
||||
@@ -1,4 +1,7 @@
|
||||
package com.zdjizhi.utils;
|
||||
import cn.hutool.log.Log;
|
||||
import cn.hutool.log.LogFactory;
|
||||
import com.zdjizhi.common.CommonConfig;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
@@ -8,48 +11,69 @@ import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
|
||||
public class HdfsUtils {
|
||||
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(HdfsUtils.class);
|
||||
|
||||
private static final Log logger = LogFactory.get();
|
||||
private static FileSystem fileSystem;
|
||||
|
||||
static {
|
||||
Configuration configuration = new Configuration();
|
||||
try {
|
||||
//指定用户
|
||||
//配置hdfs相关信息
|
||||
// configuration.set("fs.defaultFS","hdfs://ns1");
|
||||
// configuration.set("hadoop.proxyuser.root.hosts","*");
|
||||
// configuration.set("hadoop.proxyuser.root.groups","*");
|
||||
// configuration.set("ha.zookeeper.quorum", CommonConfig.HBASE_ZOOKEEPER_QUORUM);
|
||||
// configuration.set("dfs.nameservices","ns1");
|
||||
// configuration.set("dfs.ha.namenodes.ns1","nn1,nn2");
|
||||
// configuration.set("dfs.namenode.rpc-address.ns1.nn1",CommonConfig.HDFS_URI_NS1);
|
||||
// configuration.set("dfs.namenode.rpc-address.ns1.nn2",CommonConfig.HDFS_URI_NS2);
|
||||
// configuration.set("dfs.client.failover.proxy.provider.ns1","org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider");
|
||||
// //创建fileSystem,用于连接hdfs
|
||||
//// fileSystem = FileSystem.get(configuration);
|
||||
System.setProperty("HADOOP_USER_NAME", CommonConfig.HDFS_USER);
|
||||
//创建fileSystem,用于连接hdfs
|
||||
fileSystem = FileSystem.get(configuration);
|
||||
fileSystem = FileSystem.get(new URI(CommonConfig.HDFS_URI_NS1),configuration);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
catch (URISyntaxException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
|
||||
public static boolean isExists(String filePath) throws IOException {
|
||||
return fileSystem.exists(new Path(filePath));
|
||||
}
|
||||
|
||||
public static byte[] getFileBytes(String filePath) throws IOException {
|
||||
FSDataInputStream open = null;
|
||||
try {
|
||||
open = fileSystem.open(new Path(filePath));
|
||||
try (FSDataInputStream open = fileSystem.open(new Path(filePath))) {
|
||||
byte[] bytes = new byte[open.available()];
|
||||
open.read(0,bytes,0, open.available());
|
||||
open.read(0, bytes, 0, open.available());
|
||||
return bytes;
|
||||
} finally {
|
||||
if (open != null) {
|
||||
open.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
logger.error("An I/O exception when files are download from HDFS. Message is :" + e.getMessage());
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public static void uploadFileByBytes(String filePath,byte[] bytes) throws IOException {
|
||||
try (FSDataOutputStream fsDataOutputStream = fileSystem.create(new Path(filePath), true)) {
|
||||
fsDataOutputStream.write(bytes);
|
||||
fsDataOutputStream.flush();
|
||||
} catch (RuntimeException e) {
|
||||
logger.error("Uploading files to the HDFS is abnormal. Message is :" + e.getMessage());
|
||||
} catch (IOException e) {
|
||||
logger.error("An I/O exception when files are uploaded to HDFS. Message is :" + e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public static void uploadFileByBytes(String filePath, byte[] bytes) throws IOException {
|
||||
FSDataOutputStream fsDataOutputStream = null;
|
||||
try {
|
||||
fsDataOutputStream = fileSystem.create(new Path(filePath), true);
|
||||
fsDataOutputStream.write(bytes);
|
||||
} finally {
|
||||
if (fsDataOutputStream != null) {
|
||||
fsDataOutputStream.close();
|
||||
}
|
||||
}
|
||||
public static void rename(String src, String dst) throws IOException {
|
||||
fileSystem.rename(new Path(src),new Path(dst));
|
||||
}
|
||||
|
||||
|
||||
|
||||
@@ -1,139 +0,0 @@
|
||||
package com.zdjizhi.utils;
|
||||
|
||||
import com.alibaba.nacos.api.NacosFactory;
|
||||
import com.alibaba.nacos.api.PropertyKeyConst;
|
||||
import com.alibaba.nacos.api.config.ConfigService;
|
||||
import com.alibaba.nacos.api.config.listener.Listener;
|
||||
import com.fasterxml.jackson.databind.JavaType;
|
||||
import com.zdjizhi.common.CommonConfig;
|
||||
import com.zdjizhi.common.CustomFile;
|
||||
import com.zdjizhi.common.KnowledgeLog;
|
||||
//import com.zdjizhi.function.source.RichHttpSourceFunction;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.http.Header;
|
||||
import org.apache.http.message.BasicHeader;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
|
||||
public class HttpSource extends RichHttpSourceFunction<List<CustomFile>> {
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(HttpSource.class);
|
||||
|
||||
//连接nacos的配置
|
||||
private Properties nacosProperties;
|
||||
|
||||
//nacos data id
|
||||
private String NACOS_DATA_ID;
|
||||
|
||||
//nacos group
|
||||
private String NACOS_GROUP;
|
||||
|
||||
//nacos 连接超时时间
|
||||
private long NACOS_READ_TIMEOUT;
|
||||
|
||||
//上传到hdfs的路径
|
||||
private String STORE_PATH;
|
||||
|
||||
private ConfigService configService;
|
||||
|
||||
private static JsonMapper jsonMapperInstance = JsonMapper.getInstance();
|
||||
private static JavaType listType = jsonMapperInstance.createCollectionType(List.class, KnowledgeLog.class);
|
||||
|
||||
private boolean isRunning = true;
|
||||
|
||||
|
||||
public HttpSource(Properties nacosProperties, String NACOS_DATA_ID, String NACOS_GROUP, long NACOS_READ_TIMEOUT, String storePath) {
|
||||
this.nacosProperties = nacosProperties;
|
||||
this.NACOS_DATA_ID = NACOS_DATA_ID;
|
||||
this.NACOS_GROUP = NACOS_GROUP;
|
||||
this.NACOS_READ_TIMEOUT = NACOS_READ_TIMEOUT;
|
||||
this.STORE_PATH = storePath;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
logger.info("连接nacos:" + nacosProperties.getProperty(PropertyKeyConst.SERVER_ADDR));
|
||||
configService = NacosFactory.createConfigService(nacosProperties);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run(SourceContext ctx) throws Exception {
|
||||
String config = configService.getConfig(NACOS_DATA_ID, NACOS_GROUP, NACOS_READ_TIMEOUT);
|
||||
logger.info("receive config from nacos:" + config);
|
||||
System.out.println("receive config from nacos:" + config);
|
||||
List<CustomFile> customFiles;
|
||||
//从nacos拿到配置后连接hos下载知识库到内存
|
||||
customFiles = loadKnowledge(config);
|
||||
//将知识库传递到下一节点,即广播
|
||||
ctx.collectWithTimestamp(customFiles,System.currentTimeMillis());
|
||||
|
||||
configService.addListener(NACOS_DATA_ID, NACOS_GROUP, new Listener() {
|
||||
@Override
|
||||
public Executor getExecutor() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receiveConfigInfo(String configMsg) {
|
||||
try {
|
||||
logger.info("receive update config:" + configMsg);
|
||||
List<CustomFile> customFiles = new ArrayList<>();
|
||||
customFiles = loadKnowledge(configMsg);
|
||||
ctx.collectWithTimestamp(customFiles,System.currentTimeMillis());
|
||||
//将更新后的文件重新上传至hdfs
|
||||
for (CustomFile customFile : customFiles) {
|
||||
logger.info("begin upload to hdfs:" + STORE_PATH + customFile.getFileName());
|
||||
HdfsUtils.uploadFileByBytes(STORE_PATH + customFile.getFileName(),customFile.getContent());
|
||||
logger.info(STORE_PATH + customFile.getFileName() + " upload finished");
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.error("监听nacos配置失败", e);
|
||||
}
|
||||
System.out.println(configMsg);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
//下载知识库到内存
|
||||
public List<CustomFile> loadKnowledge(String config) {
|
||||
List<CustomFile> customFiles = new ArrayList<>();
|
||||
List<KnowledgeLog> knowledges = jsonMapperInstance.fromJson(config, listType);
|
||||
for (KnowledgeLog knowledge : knowledges) {
|
||||
CustomFile customFile = new CustomFile();
|
||||
String fileName = knowledge.getName().concat(".").concat(knowledge.getFormat());
|
||||
customFile.setFileName(fileName);
|
||||
InputStream inputStream = null;
|
||||
try {
|
||||
Header header = new BasicHeader("token", CommonConfig.HOS_TOKEN);
|
||||
HttpClientUtils httpClientUtils = new HttpClientUtils();
|
||||
inputStream = httpClientUtils.httpGetInputStream(knowledge.getPath(), 3000, header);
|
||||
byte[] bytes = IOUtils.toByteArray(inputStream);
|
||||
customFile.setContent(bytes);
|
||||
} catch (IOException ioException) {
|
||||
ioException.printStackTrace();
|
||||
} finally {
|
||||
IOUtils.closeQuietly(inputStream);
|
||||
}
|
||||
customFiles.add(customFile);
|
||||
}
|
||||
return customFiles;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cancel() {
|
||||
this.isRunning = false;
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -1,18 +1,104 @@
|
||||
package com.zdjizhi.utils;
|
||||
|
||||
import com.zdjizhi.common.CommonConfig;
|
||||
import com.zdjizhi.common.CustomFile;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class IpUtils {
|
||||
public static IpLookupV2 ipLookup ;
|
||||
|
||||
private static Logger LOG = LoggerFactory.getLogger(IpUtils.class);
|
||||
|
||||
|
||||
/**
|
||||
* IP定位库工具类
|
||||
*/
|
||||
public static IpLookupV2 ipLookup = new IpLookupV2.Builder(false)
|
||||
.loadDataFileV4(CommonConfig.IP_MMDB_PATH + "ip_v4_built_in.mmdb")
|
||||
.loadDataFileV6(CommonConfig.IP_MMDB_PATH + "ip_v6_built_in.mmdb")
|
||||
.loadDataFilePrivateV4(CommonConfig.IP_MMDB_PATH + "ip_v4_user_defined.mmdb")
|
||||
.loadDataFilePrivateV6(CommonConfig.IP_MMDB_PATH + "ip_v6_user_defined.mmdb")
|
||||
.build();
|
||||
// public static IpLookupV2 ipLookup = new IpLookupV2.Builder(false)
|
||||
// .loadDataFileV4(CommonConfig.IP_MMDB_PATH + "ip_v4_built_in.mmdb")
|
||||
// .loadDataFileV6(CommonConfig.IP_MMDB_PATH + "ip_v6_built_in.mmdb")
|
||||
// .loadDataFilePrivateV4(CommonConfig.IP_MMDB_PATH + "ip_v4_user_defined.mmdb")
|
||||
// .loadDataFilePrivateV6(CommonConfig.IP_MMDB_PATH + "ip_v6_user_defined.mmdb")
|
||||
// .build();
|
||||
|
||||
|
||||
public static void loadIpLook(){
|
||||
try {
|
||||
IpLookupV2.Builder builder = new IpLookupV2.Builder(false);
|
||||
if ("CLUSTER".equals(CommonConfig.CLUSTER_OR_SINGLE)) {
|
||||
byte[] ipv4BuiltBytes = HdfsUtils.getFileBytes(CommonConfig.HDFS_PATH + "ip_v4_built_in.mmdb");
|
||||
if (ipv4BuiltBytes!=null){
|
||||
InputStream ipv4BuiltInputStream = new ByteArrayInputStream(ipv4BuiltBytes);
|
||||
builder.loadDataFileV4(ipv4BuiltInputStream);
|
||||
}
|
||||
|
||||
byte[] ipv6BuiltBytes = HdfsUtils.getFileBytes(CommonConfig.HDFS_PATH + "ip_v6_built_in.mmdb");
|
||||
if (ipv6BuiltBytes!=null){
|
||||
InputStream ipv6BuiltInputStream = new ByteArrayInputStream(ipv6BuiltBytes);
|
||||
builder.loadDataFileV6(ipv6BuiltInputStream);
|
||||
}
|
||||
|
||||
byte[] ipv4UserBytes = HdfsUtils.getFileBytes(CommonConfig.HDFS_PATH + "ip_v4_user_defined.mmdb");
|
||||
if (ipv4UserBytes!=null){
|
||||
InputStream ipv4UserInputStream = new ByteArrayInputStream(ipv4UserBytes);
|
||||
builder.loadDataFilePrivateV4(ipv4UserInputStream);
|
||||
}
|
||||
|
||||
byte[] ipv6UserBytes = HdfsUtils.getFileBytes(CommonConfig.HDFS_PATH + "ip_v6_user_defined.mmdb");
|
||||
if (ipv6UserBytes!=null){
|
||||
InputStream ipv6UserInputStream = new ByteArrayInputStream(ipv6UserBytes);
|
||||
builder.loadDataFilePrivateV6(ipv6UserInputStream);
|
||||
}
|
||||
}else if ("SINGLE".equals(CommonConfig.CLUSTER_OR_SINGLE)){
|
||||
byte[] ipv4BuiltBytes = FileByteUtils.getFileBytes(CommonConfig.DOWNLOAD_PATH + "ip_v4_built_in.mmdb");
|
||||
if (ipv4BuiltBytes!=null){
|
||||
InputStream ipv4BuiltInputStream = new ByteArrayInputStream(ipv4BuiltBytes);
|
||||
builder.loadDataFileV4(ipv4BuiltInputStream);
|
||||
}
|
||||
|
||||
byte[] ipv6BuiltBytes = FileByteUtils.getFileBytes(CommonConfig.DOWNLOAD_PATH + "ip_v6_built_in.mmdb");
|
||||
if (ipv6BuiltBytes!=null){
|
||||
InputStream ipv6BuiltInputStream = new ByteArrayInputStream(ipv6BuiltBytes);
|
||||
builder.loadDataFileV6(ipv6BuiltInputStream);
|
||||
}
|
||||
|
||||
byte[] ipv4UserBytes = FileByteUtils.getFileBytes(CommonConfig.DOWNLOAD_PATH + "ip_v4_user_defined.mmdb");
|
||||
if (ipv4UserBytes!=null){
|
||||
InputStream ipv4UserInputStream = new ByteArrayInputStream(ipv4UserBytes);
|
||||
builder.loadDataFilePrivateV4(ipv4UserInputStream);
|
||||
}
|
||||
|
||||
byte[] ipv6UserBytes = FileByteUtils.getFileBytes(CommonConfig.DOWNLOAD_PATH + "ip_v6_user_defined.mmdb");
|
||||
if (ipv6UserBytes!=null){
|
||||
InputStream ipv6UserInputStream = new ByteArrayInputStream(ipv6UserBytes);
|
||||
builder.loadDataFilePrivateV6(ipv6UserInputStream);
|
||||
}
|
||||
}
|
||||
ipLookup = builder.build();
|
||||
|
||||
}catch (Exception e){
|
||||
LOG.error("加载失败",e);
|
||||
}
|
||||
}
|
||||
|
||||
public static void updateIpLook(Map<String, byte[]> knowledgeFileCache){
|
||||
try{
|
||||
IpLookupV2.Builder builder = new IpLookupV2.Builder(false);
|
||||
ipLookup= builder.loadDataFileV4(new ByteArrayInputStream(knowledgeFileCache.get("ip_v4_built_in.mmdb")))
|
||||
.loadDataFileV6(new ByteArrayInputStream(knowledgeFileCache.get("ip_v6_built_in.mmdb")))
|
||||
.loadDataFilePrivateV4(new ByteArrayInputStream(knowledgeFileCache.get("ip_v4_user_defined.mmdb")))
|
||||
.loadDataFilePrivateV6(new ByteArrayInputStream(knowledgeFileCache.get("ip_v6_user_defined.mmdb")))
|
||||
.build();
|
||||
}catch (Exception e){
|
||||
LOG.error("加载失败",e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static void main(String[] args) {
|
||||
System.out.println(ipLookup.countryLookup("49.7.115.37"));
|
||||
|
||||
@@ -1,32 +0,0 @@
|
||||
package com.zdjizhi.utils;
|
||||
import lombok.Data;
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2022/10/19 18:27
|
||||
*/
|
||||
|
||||
|
||||
@Data
|
||||
public class KnowledgeBase {
|
||||
|
||||
private String id;
|
||||
|
||||
private String name;
|
||||
|
||||
private String type;
|
||||
|
||||
private String path;
|
||||
|
||||
private Long size;
|
||||
|
||||
private String format;
|
||||
|
||||
private String sha256;
|
||||
|
||||
private String origin_url;
|
||||
|
||||
private String version;
|
||||
|
||||
private String updateTime;
|
||||
}
|
||||
@@ -1,24 +0,0 @@
|
||||
package com.zdjizhi.utils;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2022/10/19 18:25
|
||||
* 提取了下知识库的名称
|
||||
*/
|
||||
public class KnowledgeConstant {
|
||||
|
||||
public static final String DAT = "dat";
|
||||
public static final String IP_USER_DEFINED_V4 = "ip_v4_user_defined";
|
||||
public static final String IP_USER_DEFINED_V6 = "ip_v6_user_defined";
|
||||
public static final String IP_BUILT_IN_V4 = "ip_v4_built_in";
|
||||
public static final String IP_BUILT_IN_V6 = "ip_v6_built_in";
|
||||
public static final String IP_USER_DEFINED_V4_MMDB = "ip_v4_user_defined.mmdb";
|
||||
public static final String IP_USER_DEFINED_V6_MMDB = "ip_v6_user_defined.mmdb";
|
||||
public static final String IP_BUILT_IN_V4_MMDB = "ip_v4_built_in.mmdb";
|
||||
public static final String IP_BUILT_IN_V6_MMDB = "ip_v6_built_in.mmdb";
|
||||
public static final String LATEST = "latest";
|
||||
public static final String TOKEN = "token";
|
||||
|
||||
}
|
||||
|
||||
@@ -1,95 +0,0 @@
|
||||
package com.zdjizhi.utils;
|
||||
|
||||
import cn.hutool.core.io.IoUtil;
|
||||
import cn.hutool.http.Header;
|
||||
import com.fasterxml.jackson.databind.JavaType;
|
||||
import org.apache.http.message.BasicHeader;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.kafka.common.requests.FetchMetadata.log;
|
||||
|
||||
/**
|
||||
* @author fy
|
||||
* @version 1.0
|
||||
* @date 2022/10/19 14:07
|
||||
*/
|
||||
public class KnowledgeUtils {
|
||||
|
||||
private static JsonMapper jsonMapperInstance = JsonMapper.getInstance();
|
||||
private static JavaType listType = jsonMapperInstance.createCollectionType(List.class, KnowledgeBase.class);
|
||||
|
||||
|
||||
|
||||
|
||||
public Map<String, KnowledgeBase> queryLoadKnowledgeInfo(String knowledgeMetadata, IpLocationConfiguration ipLocationConfiguration) {
|
||||
|
||||
log.info("update knowledge Base version:{},update knowledge Base content:{}", ipLocationConfiguration, knowledgeMetadata);
|
||||
Map<String, KnowledgeBase> map = new HashMap<>();
|
||||
|
||||
if (StringUtil.isNotEmpty(ipLocationConfiguration) && StringUtil.isNotEmpty(knowledgeMetadata)) {
|
||||
// List<KnowledgeBase> knowledgeBaseList = JsonMapper.fromJsonString(knowledgeMetadata, List.class, KnowledgeBase.class);
|
||||
|
||||
List<KnowledgeBase> knowledgeBaseList = jsonMapperInstance.fromJson(knowledgeMetadata, listType);// TODO: 2022/10/21
|
||||
|
||||
String ipV4BuiltIn = ipLocationConfiguration.getIpV4BuiltIn();
|
||||
String ipV4UserDefined = ipLocationConfiguration.getIpV4UserDefined();
|
||||
String ipV6BuiltIn = ipLocationConfiguration.getIpV6BuiltIn();
|
||||
String ipV6UserDefined = ipLocationConfiguration.getIpV6UserDefined();
|
||||
|
||||
for (KnowledgeBase knowledgeBase : knowledgeBaseList) {
|
||||
String name = knowledgeBase.getName();
|
||||
String version = knowledgeBase.getVersion();
|
||||
String concat = name.concat(":").concat(version);
|
||||
|
||||
if (StringUtil.equals(concat, ipV4BuiltIn)
|
||||
|| (StringUtil.equals(name, ipV4BuiltIn) && StringUtil.equals(version, KnowledgeConstant.LATEST))) {
|
||||
map.put(KnowledgeConstant.IP_BUILT_IN_V4, knowledgeBase);
|
||||
} else if (StringUtil.equals(concat, ipV4UserDefined)
|
||||
|| (StringUtil.equals(name, ipV4UserDefined) && StringUtil.equals(version, KnowledgeConstant.LATEST))) {
|
||||
map.put(KnowledgeConstant.IP_USER_DEFINED_V4, knowledgeBase);
|
||||
} else if (StringUtil.equals(concat, ipV6BuiltIn)
|
||||
|| (StringUtil.equals(name, ipV6BuiltIn) && StringUtil.equals(version, KnowledgeConstant.LATEST))) {
|
||||
map.put(KnowledgeConstant.IP_BUILT_IN_V6, knowledgeBase);
|
||||
} else if (StringUtil.equals(concat, ipV6UserDefined)
|
||||
|| (StringUtil.equals(name, ipV6UserDefined) && StringUtil.equals(version, KnowledgeConstant.LATEST))) {
|
||||
map.put(KnowledgeConstant.IP_USER_DEFINED_V6, knowledgeBase);
|
||||
}
|
||||
}
|
||||
}
|
||||
return map;
|
||||
}
|
||||
|
||||
|
||||
private void download(KnowledgeBase knowledgeBase) {
|
||||
|
||||
String id = knowledgeBase.getId();
|
||||
String name = knowledgeBase.getName();
|
||||
String sha256 = knowledgeBase.getSha256();
|
||||
String format = knowledgeBase.getFormat();
|
||||
FileOutputStream outputStream = null;
|
||||
InputStream inputStream = null;
|
||||
try {
|
||||
Header header = new BasicHeader(KnowledgeConstant.TOKEN, hosConfig.getToken());
|
||||
inputStream = httpClientService.httpGetInputStream(knowledgeBase.getPath(), httpConfig.getServerResponseTimeOut(), header);
|
||||
File file = new File(KnowledgeConstant.DAT.concat(File.separator).concat(name).concat(".").concat(format));
|
||||
outputStream = new FileOutputStream(file);
|
||||
IoUtil.copy(inputStream, outputStream);
|
||||
log.info("knowledge download name :{},version:{}", name, knowledgeBase.getVersion());
|
||||
} catch (IOException e) {
|
||||
log.error("file not fount,:{}", e);
|
||||
} finally {
|
||||
IoUtil.close(inputStream);
|
||||
IoUtil.close(outputStream);
|
||||
}
|
||||
updateMap.put(id, sha256);
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -1,75 +0,0 @@
|
||||
package com.zdjizhi.etl.utils;
|
||||
|
||||
import com.opencsv.CSVParser;
|
||||
import com.opencsv.CSVReader;
|
||||
import com.zdjizhi.base.common.CnRecordLog;
|
||||
import com.zdjizhi.base.common.CommonConfig;
|
||||
import com.zdjizhi.etl.common.LinkAndIspInfo;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.DataInputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.HashMap;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* @ Author pengfeixu
|
||||
* @ Date 2022/3/30
|
||||
* @ Description
|
||||
*/
|
||||
public class LinkUtils {
|
||||
private static Logger LOG = LoggerFactory.getLogger(LinkUtils.class);
|
||||
public static HashMap<Long, LinkAndIspInfo> linkmap = new HashMap<>();
|
||||
// private static NacosUtils nacosUtils = new NacosUtils();
|
||||
// private static HashMap<Long, LinkAndIspInfo> linkmap = readLinkCsv(CommonConfig.LINK_PATH);
|
||||
|
||||
public static void readLinkCsv(String path) {
|
||||
HashMap<Long, LinkAndIspInfo> linkDirectionmap = new HashMap<>();
|
||||
|
||||
try {
|
||||
DataInputStream in = new DataInputStream(new FileInputStream(new File(path)));
|
||||
CSVReader csvReader = new CSVReader(new InputStreamReader(in, StandardCharsets.UTF_8), CSVParser.DEFAULT_SEPARATOR,
|
||||
CSVParser.DEFAULT_QUOTE_CHARACTER, CSVParser.DEFAULT_ESCAPE_CHARACTER, 1);
|
||||
|
||||
for (String[] strs : csvReader) {
|
||||
long key=Long.parseLong(strs[6]);
|
||||
LinkAndIspInfo linkDirectionInfo = new LinkAndIspInfo();
|
||||
linkDirectionInfo.setEgress_link_direction(strs[3]);
|
||||
linkDirectionInfo.setIngress_link_direction(strs[3]);
|
||||
linkDirectionInfo.setLink_id(Integer.parseInt(strs[6]));
|
||||
linkmap.put(key, linkDirectionInfo);
|
||||
}
|
||||
csvReader.close();
|
||||
} catch (Exception e) {
|
||||
LOG.error(e.getMessage());
|
||||
}
|
||||
// return linkDirectionmap;
|
||||
}
|
||||
|
||||
public static void setLinkMessage(CnRecordLog cnRecordLog) {
|
||||
if (! linkmap.containsKey(cnRecordLog.getCommon_egress_link_id()) || ! linkmap.containsKey(cnRecordLog.getCommon_ingress_link_id())){
|
||||
cnRecordLog.setCommon_egress_link_id(0);
|
||||
cnRecordLog.setCommon_ingress_link_id(0);
|
||||
}
|
||||
|
||||
if (linkmap.containsKey(cnRecordLog.getCommon_egress_link_id())){
|
||||
cnRecordLog.setEgress_link_direction(linkmap.get(cnRecordLog.getCommon_egress_link_id()).getEgress_link_direction());
|
||||
}
|
||||
if (linkmap.containsKey(cnRecordLog.getCommon_ingress_link_id())){
|
||||
cnRecordLog.setIngress_link_direction(linkmap.get(cnRecordLog.getCommon_ingress_link_id()).getIngress_link_direction());
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) {
|
||||
// HashMap<Long, LinkAndIspInfo> map = readLinkCsv(CommonConfig.LINK_PATH);
|
||||
Set<Long> keySet = linkmap.keySet();
|
||||
for (long key : keySet){
|
||||
System.out.println(linkmap.get(key).toString());
|
||||
}
|
||||
System.out.println(linkmap.size());
|
||||
}
|
||||
}
|
||||
@@ -19,11 +19,11 @@ public class NacosUtils {
|
||||
|
||||
|
||||
private static final String NACOS_SERVER_ADDR = CommonConfigurations.getStringProperty("nacos.server.addr");
|
||||
private static final String NACOS_NAMESPACE = CommonConfigurations.getStringProperty("nacos.namespace");
|
||||
private static final String NACOS_STATIC_NAMESPACE = CommonConfigurations.getStringProperty("nacos.static.namespace");
|
||||
private static final String NACOS_USERNAME = CommonConfigurations.getStringProperty("nacos.username");
|
||||
private static final String NACOS_PASSWORD = CommonConfigurations.getStringProperty("nacos.password");
|
||||
private static final String NACOS_DATA_ID = CommonConfigurations.getStringProperty("nacos.data.id");
|
||||
private static final String NACOS_GROUP = CommonConfigurations.getStringProperty("nacos.group");
|
||||
private static final String NACOS_STATIC_DATA_ID = CommonConfigurations.getStringProperty("nacos.static.data.id");
|
||||
private static final String NACOS_STATIC_GROUP = CommonConfigurations.getStringProperty("nacos.static.group");
|
||||
private static final long NACOS_READ_TIMEOUT = CommonConfigurations.getLongProperty("nacos.read.timeout");
|
||||
|
||||
static {
|
||||
@@ -32,7 +32,7 @@ public class NacosUtils {
|
||||
|
||||
private static void getProperties() {
|
||||
nacosProperties.setProperty(PropertyKeyConst.SERVER_ADDR, NACOS_SERVER_ADDR);
|
||||
nacosProperties.setProperty(PropertyKeyConst.NAMESPACE, NACOS_NAMESPACE);
|
||||
nacosProperties.setProperty(PropertyKeyConst.NAMESPACE, NACOS_STATIC_NAMESPACE);
|
||||
nacosProperties.setProperty(PropertyKeyConst.USERNAME, NACOS_USERNAME);
|
||||
nacosProperties.setProperty(PropertyKeyConst.PASSWORD, NACOS_PASSWORD);
|
||||
}
|
||||
@@ -41,11 +41,11 @@ public class NacosUtils {
|
||||
try {
|
||||
getProperties();
|
||||
ConfigService configService = NacosFactory.createConfigService(nacosProperties);
|
||||
String config = configService.getConfig(NACOS_DATA_ID, NACOS_GROUP, NACOS_READ_TIMEOUT);
|
||||
String config = configService.getConfig(NACOS_STATIC_DATA_ID, NACOS_STATIC_GROUP, NACOS_READ_TIMEOUT);
|
||||
commonProperties.load(new StringReader(config));
|
||||
|
||||
|
||||
configService.addListener(NACOS_DATA_ID, NACOS_GROUP, new Listener() {
|
||||
configService.addListener(NACOS_STATIC_DATA_ID, NACOS_STATIC_GROUP, new Listener() {
|
||||
@Override
|
||||
public Executor getExecutor() {
|
||||
return null;
|
||||
|
||||
@@ -1,352 +0,0 @@
|
||||
package com.zdjizhi.utils;
|
||||
|
||||
import cn.hutool.core.io.IoUtil;
|
||||
import com.alibaba.nacos.api.NacosFactory;
|
||||
import com.alibaba.nacos.api.PropertyKeyConst;
|
||||
import com.alibaba.nacos.api.config.ConfigService;
|
||||
import com.alibaba.nacos.api.config.listener.Listener;
|
||||
import com.fasterxml.jackson.databind.JavaType;
|
||||
//import com.zdjizhi.base.common.CommonConfig;
|
||||
//import com.zdjizhi.base.common.KnowledgeConstant;
|
||||
//import com.zdjizhi.base.common.KnowledgeLog;
|
||||
//import com.zdjizhi.base.utils.CommonConfigurations;
|
||||
//import com.zdjizhi.base.utils.HttpClientUtils;
|
||||
import com.zdjizhi.utils.*;
|
||||
import org.apache.http.Header;
|
||||
import org.apache.http.message.BasicHeader;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
|
||||
public class NacosUtils2 {
|
||||
|
||||
|
||||
private static final Logger logger = LoggerFactory.getLogger(NacosUtils2.class);
|
||||
private static CommonConfigurations configurations;
|
||||
|
||||
|
||||
|
||||
|
||||
private static Properties nacosProperties = new Properties();
|
||||
private static Properties commonProperties = new Properties();
|
||||
|
||||
private static Map<String, String> updateMap = new HashMap<>();//存文件路径(放在服务器了)
|
||||
|
||||
private static JsonMapper jsonMapperInstance = JsonMapper.getInstance();
|
||||
private static JavaType listType = jsonMapperInstance.createCollectionType(List.class, KnowledgeLog.class);
|
||||
|
||||
private static final String NACOS_SERVER_ADDR = configurations.getStringProperty("nacos.server.addr");
|
||||
private static final String NACOS_NAMESPACE = configurations.getStringProperty("nacos.namespace");
|
||||
private static final String NACOS_USERNAME = configurations.getStringProperty("nacos.username");
|
||||
private static final String NACOS_PASSWORD = configurations.getStringProperty("nacos.password");
|
||||
private static final String NACOS_DATA_ID = configurations.getStringProperty("nacos.data.id");
|
||||
private static final String NACOS_GROUP = configurations.getStringProperty("nacos.group");
|
||||
private static final long NACOS_READ_TIMEOUT = configurations.getLongProperty("nacos.read.timeout");
|
||||
|
||||
static {
|
||||
Properties propService;
|
||||
try {
|
||||
propService = new Properties();
|
||||
propService.load(CommonConfigurations.class.getClassLoader().getResourceAsStream("common.properties"));
|
||||
configurations = new CommonConfigurations(propService);
|
||||
} catch (Exception e) {
|
||||
logger.error("加载common.properties配置文件失败");
|
||||
System.exit(1);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
static {
|
||||
createConfigService();
|
||||
}
|
||||
|
||||
private static void getProperties() {
|
||||
nacosProperties.setProperty(PropertyKeyConst.SERVER_ADDR, NACOS_SERVER_ADDR);
|
||||
nacosProperties.setProperty(PropertyKeyConst.NAMESPACE, NACOS_NAMESPACE);
|
||||
nacosProperties.setProperty(PropertyKeyConst.USERNAME, NACOS_USERNAME);
|
||||
nacosProperties.setProperty(PropertyKeyConst.PASSWORD, NACOS_PASSWORD);
|
||||
}
|
||||
|
||||
public static void createConfigService() {
|
||||
try {
|
||||
getProperties();
|
||||
ConfigService configService = NacosFactory.createConfigService(nacosProperties);
|
||||
String config = configService.getConfig(NACOS_DATA_ID, NACOS_GROUP, NACOS_READ_TIMEOUT);
|
||||
// commonProperties.load(new StringReader(config));
|
||||
|
||||
// Map<String, String> pathMap =
|
||||
loadKnowledge(config);;
|
||||
|
||||
configService.addListener(NACOS_DATA_ID, NACOS_GROUP, new Listener() {
|
||||
@Override
|
||||
public Executor getExecutor() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void receiveConfigInfo(String configMsg) {
|
||||
try {
|
||||
// commonProperties.clear();
|
||||
// commonProperties.load(new StringReader(configMsg));
|
||||
loadKnowledge(configMsg);
|
||||
// IdcRenterUtils.readServerCsv(configMsg);
|
||||
} catch (Exception e) {
|
||||
logger.error("监听nacos配置失败", e);
|
||||
}
|
||||
System.out.println(configMsg);
|
||||
}
|
||||
});
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
logger.error("获取nacos配置失败", e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
//加载nacos配置
|
||||
public static void loadKnowledge(String config){
|
||||
Map<String, KnowledgeLog> knowledgeLogMap = queryLoadKnowledgeInfo(config);
|
||||
|
||||
|
||||
for (Map.Entry<String, KnowledgeLog> knowledgeLogEntry : knowledgeLogMap.entrySet()) {
|
||||
KnowledgeLog knowledgeLog = knowledgeLogEntry.getValue();
|
||||
String id = knowledgeLog.getId();
|
||||
String sha256 = knowledgeLog.getSha256();
|
||||
if (!StringUtil.equals(sha256,updateMap.get(id))){
|
||||
System.out.println(knowledgeLogEntry.getValue());
|
||||
download( knowledgeLogEntry.getValue());//下载文件
|
||||
}
|
||||
}
|
||||
Map<String, String> path = queryPath(knowledgeLogMap);
|
||||
|
||||
//初始化ip定位库
|
||||
IpUtils.initIpLookup(path.get(KnowledgeConstant.IP_BUILT_IN_V4_MMDB),path.get(KnowledgeConstant.ASN_V4),path.get(KnowledgeConstant.ASN_V6));
|
||||
IdcRenterUtils.readServerCsv(path.get(KnowledgeConstant.ISP));
|
||||
LinkUtils.readLinkCsv(path.get(KnowledgeConstant.LINK));
|
||||
IspUtils.readServerCsv(path.get(KnowledgeConstant.ISP));
|
||||
FcUtils.readCsv(path.get(KnowledgeConstant.WEBSKT));
|
||||
FcUtils.readIcpCsv(path.get(KnowledgeConstant.ICP));
|
||||
WhoisUtils.readJson(path.get(KnowledgeConstant.WHOIS));
|
||||
DnsServerUtils.readServerCsv(path.get(KnowledgeConstant.DNSPATH));
|
||||
AppUtils.readJson(path.get(KnowledgeConstant.APPSKT));
|
||||
IPUtils.readInternalIpCsv(path.get(KnowledgeConstant.INTERNALIP));
|
||||
|
||||
}
|
||||
|
||||
//query知识库中json信息
|
||||
public static Map<String,KnowledgeLog> queryLoadKnowledgeInfo(String content){
|
||||
// KnowledgeLog knowledgeLog = new KnowledgeLog();
|
||||
Map<String, KnowledgeLog> map = new HashMap<>();
|
||||
|
||||
// ArrayList<HashMap<String, Object>> o = jsonMapperInstance.fromJson(content, listType);
|
||||
List<KnowledgeLog> o = jsonMapperInstance.fromJson(content, listType);
|
||||
for (KnowledgeLog knowledgeLog : o) {
|
||||
|
||||
// }
|
||||
// for (HashMap<String, Object> obj : o) {
|
||||
// knowledgeLog.setId(obj.get("id").toString());
|
||||
// knowledgeLog.setName(obj.get("name").toString());
|
||||
// knowledgeLog.setPath(obj.get("path").toString());
|
||||
// knowledgeLog.setSize(Long.parseLong(obj.get("size").toString()));
|
||||
// knowledgeLog.setFormat(obj.get("format").toString());
|
||||
// knowledgeLog.setSha256(obj.get("sha256").toString());
|
||||
// knowledgeLog.setVersion(obj.get("version").toString());
|
||||
// knowledgeLog.setUpdateTime(obj.get("updateTime").toString());
|
||||
|
||||
if ((KnowledgeConstant.IP_BUILT_IN_V4_MMDB.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat()))) && (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.IP_BUILT_IN_V4_MMDB,knowledgeLog);
|
||||
}
|
||||
if (((KnowledgeConstant.ASN_V4.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat()))) )&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.ASN_V4,knowledgeLog);
|
||||
}
|
||||
if ((KnowledgeConstant.ASN_V6.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat())))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion())) ){
|
||||
map.put(KnowledgeConstant.ASN_V6,knowledgeLog);
|
||||
}
|
||||
if ((KnowledgeConstant.WEBSKT.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat()))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion())) )){
|
||||
map.put(KnowledgeConstant.WEBSKT,knowledgeLog);
|
||||
}
|
||||
if ((KnowledgeConstant.DNSPATH.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat())))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.DNSPATH,knowledgeLog);
|
||||
}
|
||||
if ((KnowledgeConstant.APPSKT.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat())))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.APPSKT,knowledgeLog);
|
||||
}
|
||||
if ((KnowledgeConstant.WHOIS.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat())))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.WHOIS,knowledgeLog);
|
||||
}
|
||||
if( (KnowledgeConstant.ICP.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat())))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.ICP,knowledgeLog);
|
||||
}
|
||||
if ((KnowledgeConstant.IOC.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat())))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.IOC,knowledgeLog);
|
||||
}
|
||||
if ((KnowledgeConstant.LINK.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat())))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.LINK,knowledgeLog);
|
||||
}
|
||||
if (KnowledgeConstant.ISP.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat()))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.ISP,knowledgeLog);
|
||||
}
|
||||
if (KnowledgeConstant.IDCRENTER.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat()))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.IDCRENTER,knowledgeLog);
|
||||
}
|
||||
if (KnowledgeConstant.INTERNALIP.equals(knowledgeLog.getName().concat(".").concat(knowledgeLog.getFormat()))&& (KnowledgeConstant.LATEST.equals(knowledgeLog.getVersion()))){
|
||||
map.put(KnowledgeConstant.INTERNALIP,knowledgeLog);
|
||||
}
|
||||
// }
|
||||
// download(knowledgeLog);
|
||||
}
|
||||
return map;
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
//匹配路径,文件存放在服务器的路径
|
||||
private static Map<String,String> queryPath(Map<String,KnowledgeLog> map){
|
||||
Map<String,String> pathMap=new HashMap<>();
|
||||
KnowledgeLog ip_v4 = map.get(KnowledgeConstant.IP_BUILT_IN_V4_MMDB);//获取值
|
||||
KnowledgeLog asn_v4 = map.get(KnowledgeConstant.ASN_V4);
|
||||
KnowledgeLog asn_v6 = map.get(KnowledgeConstant.ASN_V6);
|
||||
KnowledgeLog webskt = map.get(KnowledgeConstant.WEBSKT);
|
||||
KnowledgeLog dnsPath = map.get(KnowledgeConstant.DNSPATH);
|
||||
KnowledgeLog appskt = map.get(KnowledgeConstant.APPSKT);
|
||||
KnowledgeLog whois = map.get(KnowledgeConstant.WHOIS);
|
||||
KnowledgeLog icp = map.get(KnowledgeConstant.ICP);
|
||||
KnowledgeLog ioc = map.get(KnowledgeConstant.IOC);
|
||||
KnowledgeLog link = map.get(KnowledgeConstant.LINK);
|
||||
KnowledgeLog isp = map.get(KnowledgeConstant.ISP);
|
||||
KnowledgeLog idcRenter = map.get(KnowledgeConstant.IDCRENTER);
|
||||
KnowledgeLog internalIp = map.get(KnowledgeConstant.INTERNALIP);
|
||||
|
||||
if (StringUtil.isEmpty(ip_v4)|| !new File(CommonConfig.IP_PATH.concat(ip_v4.getName().concat(".").concat(ip_v4.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.IP_BUILT_IN_V4_MMDB,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.IP_BUILT_IN_V4_MMDB);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.IP_BUILT_IN_V4_MMDB,CommonConfig.IP_PATH.concat(File.separator).concat(ip_v4.getName()).concat(".").concat(ip_v4.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(asn_v4)|| !new File(CommonConfig.IP_PATH.concat(asn_v4.getName().concat(".").concat(asn_v4.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.ASN_V4,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.ASN_V4);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.ASN_V4,CommonConfig.IP_PATH.concat(File.separator).concat(asn_v4.getName()).concat(".").concat(asn_v4.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(asn_v6)|| !new File(CommonConfig.IP_PATH.concat(asn_v6.getName().concat(".").concat(asn_v6.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.ASN_V6,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.ASN_V6);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.ASN_V6,CommonConfig.IP_PATH.concat(File.separator).concat(asn_v6.getName()).concat(".").concat(asn_v6.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(webskt)|| !new File(CommonConfig.IP_PATH.concat(webskt.getName().concat(".").concat(webskt.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.WEBSKT,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.WEBSKT);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.WEBSKT,CommonConfig.IP_PATH.concat(File.separator).concat(webskt.getName()).concat(".").concat(webskt.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(dnsPath)|| !new File(CommonConfig.IP_PATH.concat(dnsPath.getName().concat(".").concat(dnsPath.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.DNSPATH,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.DNSPATH);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.DNSPATH,CommonConfig.IP_PATH.concat(File.separator).concat(dnsPath.getName()).concat(".").concat(dnsPath.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(appskt)|| !new File(CommonConfig.IP_PATH.concat(appskt.getName().concat(".").concat(appskt.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.APPSKT,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.APPSKT);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.APPSKT,CommonConfig.IP_PATH.concat(File.separator).concat(appskt.getName()).concat(".").concat(appskt.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(whois)|| !new File(CommonConfig.IP_PATH.concat(whois.getName().concat(".").concat(whois.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.WHOIS,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.WHOIS);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.WHOIS,CommonConfig.IP_PATH.concat(File.separator).concat(whois.getName()).concat(".").concat(whois.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(icp)|| !new File(CommonConfig.IP_PATH.concat(icp.getName().concat(".").concat(icp.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.ICP,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.ICP);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.ICP,CommonConfig.IP_PATH.concat(File.separator).concat(icp.getName()).concat(".").concat(icp.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(ioc)|| !new File(CommonConfig.IP_PATH.concat(ioc.getName().concat(".").concat(ioc.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.IOC,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.IOC);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.IOC,CommonConfig.IP_PATH.concat(File.separator).concat(ioc.getName()).concat(".").concat(ioc.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(link)|| !new File(CommonConfig.IP_PATH.concat(link.getName().concat(".").concat(link.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.LINK,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.LINK);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.LINK,CommonConfig.IP_PATH.concat(File.separator).concat(link.getName()).concat(".").concat(link.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(isp)|| !new File(CommonConfig.IP_PATH.concat(isp.getName().concat(".").concat(isp.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.ISP,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.ISP);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.ISP,CommonConfig.IP_PATH.concat(File.separator).concat(isp.getName()).concat(".").concat(isp.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(idcRenter)|| !new File(CommonConfig.IP_PATH.concat(idcRenter.getName().concat(".").concat(idcRenter.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.IDCRENTER,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.IDCRENTER);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.IDCRENTER,CommonConfig.IP_PATH.concat(File.separator).concat(idcRenter.getName()).concat(".").concat(idcRenter.getFormat()));
|
||||
}
|
||||
if (StringUtil.isEmpty(internalIp)|| !new File(CommonConfig.IP_PATH.concat(internalIp.getName().concat(".").concat(internalIp.getFormat()))).exists()){
|
||||
pathMap.put(KnowledgeConstant.INTERNALIP,CommonConfig.IP_PATH+File.separator+KnowledgeConstant.INTERNALIP);
|
||||
}else{
|
||||
pathMap.put(KnowledgeConstant.INTERNALIP,CommonConfig.IP_PATH.concat(File.separator).concat(internalIp.getName()).concat(".").concat(internalIp.getFormat()));
|
||||
}
|
||||
return pathMap;
|
||||
}
|
||||
|
||||
|
||||
//下载文件
|
||||
public static void download(KnowledgeLog knowledgeLog){
|
||||
String id = knowledgeLog.getId();
|
||||
String name = knowledgeLog.getName();
|
||||
String sha256 = knowledgeLog.getSha256();
|
||||
String format = knowledgeLog.getFormat();
|
||||
// String path=null;
|
||||
// Map<String,String> map=new HashMap<>();
|
||||
InputStream inputStream = null;
|
||||
FileOutputStream outputStream = null;
|
||||
|
||||
try {
|
||||
Header header = new BasicHeader("token", CommonConfig.TOKEN);
|
||||
HttpClientUtils httpClientUtils = new HttpClientUtils();
|
||||
inputStream = httpClientUtils.httpGetInputStream(knowledgeLog.getPath(), 3000, header);
|
||||
File file=new File(CommonConfig.IP_PATH.concat(File.separator).concat(knowledgeLog.getName()).concat(".").concat(knowledgeLog.getFormat()));
|
||||
outputStream=new FileOutputStream(file);
|
||||
IoUtil.copy(inputStream, outputStream);
|
||||
} catch (FileNotFoundException e) {
|
||||
e.printStackTrace();
|
||||
}finally {
|
||||
IoUtil.close(inputStream);
|
||||
IoUtil.close(outputStream);
|
||||
}
|
||||
updateMap.put(id,sha256);
|
||||
}
|
||||
|
||||
public static String getStringProperty(String key) {
|
||||
return commonProperties.getProperty(key);
|
||||
}
|
||||
|
||||
public static Integer getIntProperty(String key) {
|
||||
return Integer.parseInt(commonProperties.getProperty(key));
|
||||
}
|
||||
|
||||
public static Double getDoubleProperty(String key) {
|
||||
return Double.parseDouble(commonProperties.getProperty(key));
|
||||
}
|
||||
|
||||
public static Long getLongProperty(String key) {
|
||||
return Long.parseLong(commonProperties.getProperty(key));
|
||||
}
|
||||
|
||||
public static Boolean getBooleanProperty(String key) {
|
||||
return "true".equals(commonProperties.getProperty(key).toLowerCase().trim());
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -1,160 +0,0 @@
|
||||
package com.zdjizhi.utils;
|
||||
|
||||
|
||||
import com.alibaba.nacos.api.PropertyKeyConst;
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.maxmind.db.CHMCache;
|
||||
import com.zdjizhi.common.CustomFile;
|
||||
import org.apache.flink.api.common.state.BroadcastState;
|
||||
import org.apache.flink.api.common.state.MapStateDescriptor;
|
||||
import org.apache.flink.api.common.typeinfo.TypeInformation;
|
||||
import org.apache.flink.api.common.typeinfo.Types;
|
||||
import org.apache.flink.api.java.tuple.Tuple2;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.datastream.BroadcastConnectedStream;
|
||||
import org.apache.flink.streaming.api.datastream.BroadcastStream;
|
||||
import org.apache.flink.streaming.api.datastream.DataStreamSource;
|
||||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
|
||||
import org.apache.flink.streaming.api.functions.co.BroadcastProcessFunction;
|
||||
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
|
||||
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.InputStream;
|
||||
import java.io.Reader;
|
||||
import java.net.InetAddress;
|
||||
import java.util.*;
|
||||
|
||||
public class TestKnowledge {
|
||||
|
||||
@Test
|
||||
public void jobTest() throws Exception {
|
||||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
|
||||
|
||||
//连接nacos配置
|
||||
Properties nacosProperties = new Properties();
|
||||
//hdfs存储路径
|
||||
String STORE_PATH = "/test/";
|
||||
nacosProperties.put(PropertyKeyConst.SERVER_ADDR,"192.168.44.12");
|
||||
nacosProperties.setProperty(PropertyKeyConst.USERNAME, "nacos");
|
||||
nacosProperties.setProperty(PropertyKeyConst.PASSWORD, "nacos");
|
||||
//获取知识库的流
|
||||
DataStreamSource<List<CustomFile>> customFileDataStreamSource = env.addSource(new HttpSource(nacosProperties, "knowledge_base.json", "DEFAULT_GROUP", 30000,STORE_PATH));
|
||||
//将该流设置并行度为1,多了会产生并发拉数据情况
|
||||
customFileDataStreamSource.setParallelism(1);
|
||||
|
||||
//用于存储广播数据的state
|
||||
MapStateDescriptor<String,List<CustomFile>> descriptor =
|
||||
new MapStateDescriptor<String, List<CustomFile>>("descriptorTest", Types.STRING, Types.LIST(TypeInformation.of(CustomFile.class)));
|
||||
//将该流广播
|
||||
BroadcastStream<List<CustomFile>> broadcast = customFileDataStreamSource.broadcast(descriptor);
|
||||
|
||||
//将数据流和广播流connect
|
||||
BroadcastConnectedStream<String, List<CustomFile>> connect = env.addSource(new MySource()).setParallelism(1).connect(broadcast);
|
||||
|
||||
connect.process(new BroadcastProcessFunction<String, List<CustomFile>, Tuple2<String,String>>() {
|
||||
Reader reader = null;
|
||||
|
||||
//初始化,从hdfs拉知识库
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
System.out.println("begin init");
|
||||
byte[] fileBytes = HdfsUtils.getFileBytes(STORE_PATH + "ip_v4.mmdb");
|
||||
InputStream inputStream = new ByteArrayInputStream(fileBytes);
|
||||
this.reader = new Reader(inputStream,new CHMCache());
|
||||
System.out.println("init over");
|
||||
}
|
||||
|
||||
//数据流处理逻辑
|
||||
@Override
|
||||
public void processElement(String value, BroadcastProcessFunction<String, List<CustomFile>, Tuple2<String, String>>.ReadOnlyContext ctx, Collector<Tuple2<String, String>> out) throws Exception {
|
||||
InetAddress ipAddress = InetAddress.getByName(value);
|
||||
String result = "ip not find";
|
||||
JsonNode jsonNode = reader.get(ipAddress);
|
||||
if (jsonNode != null) {
|
||||
result = jsonNode.toString();
|
||||
}
|
||||
out.collect(Tuple2.of(value,result));
|
||||
}
|
||||
|
||||
//广播流处理逻辑,即知识库更新逻辑
|
||||
@Override
|
||||
public void processBroadcastElement(List<CustomFile> value, BroadcastProcessFunction<String, List<CustomFile>, Tuple2<String, String>>.Context ctx, Collector<Tuple2<String, String>> out) throws Exception {
|
||||
BroadcastState<String, List<CustomFile>> broadcastState = ctx.getBroadcastState(descriptor);
|
||||
broadcastState.clear();
|
||||
broadcastState.put("test",value);
|
||||
InputStream inputStream = null;
|
||||
for (CustomFile customFile : value) {
|
||||
if ("ip_v4_built_in.mmdb".equals(customFile.getFileName())) {
|
||||
inputStream = new ByteArrayInputStream(customFile.getContent());
|
||||
}
|
||||
}
|
||||
this.reader = new Reader(inputStream, new CHMCache());
|
||||
}
|
||||
}).setParallelism(1).addSink(new CollectSink1());
|
||||
|
||||
env.execute();
|
||||
|
||||
for (Tuple2<String, String> value : CollectSink1.values) {
|
||||
System.out.println(value);
|
||||
}
|
||||
System.out.println(CollectSink1.values);
|
||||
}
|
||||
|
||||
//自定义sink,用于test
|
||||
public static class CollectSink implements SinkFunction<CustomFile> {
|
||||
|
||||
public static final List<CustomFile> values = Collections.synchronizedList(new ArrayList<CustomFile>());
|
||||
|
||||
public void invoke(CustomFile value, Context context) throws Exception {
|
||||
values.add(value);
|
||||
}
|
||||
}
|
||||
|
||||
//自定义sink,用于test
|
||||
public static class CollectSink1 implements SinkFunction<Tuple2<String,String>> {
|
||||
|
||||
public static final List<Tuple2<String,String>> values = Collections.synchronizedList(new ArrayList<Tuple2<String,String>>());
|
||||
|
||||
public void invoke(Tuple2<String,String> value, Context context) throws Exception {
|
||||
//System.out.println(value.f0 + " | " + value.f1);
|
||||
values.add(value);
|
||||
}
|
||||
}
|
||||
|
||||
//自定义数据source,用于test
|
||||
public static class MySource implements SourceFunction<String> {
|
||||
|
||||
private boolean isRunning = true;
|
||||
|
||||
@Override
|
||||
public void run(SourceContext<String> ctx) throws Exception {
|
||||
System.out.println("MySource begin");
|
||||
int count = 0;
|
||||
while (isRunning) {
|
||||
Random r = new Random();
|
||||
String randomIp = r.nextInt(255) + "." + r.nextInt(255) + "." + r.nextInt(255) + "." + r.nextInt(255);
|
||||
ctx.collect(randomIp);
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cancel() {
|
||||
isRunning = false;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -22,15 +22,15 @@ kafka.input.group.id=2112080949
|
||||
kafka.output.metric.parallelism=1
|
||||
|
||||
#发送kafka metrics topic名
|
||||
kafka.output.metric.topic.name=TRAFFIC-TOP-DESTINATION-IP-METRICS
|
||||
#kafka.output.metric.topic.name=test
|
||||
#kafka.output.metric.topic.name=TRAFFIC-TOP-DESTINATION-IP-METRICS
|
||||
kafka.output.metric.topic.name=test
|
||||
|
||||
#发送kafka event并行度大小
|
||||
kafka.output.event.parallelism=1
|
||||
|
||||
#发送kafka event topic名
|
||||
kafka.output.event.topic.name=DOS-EVENT
|
||||
#kafka.output.event.topic.name=storm-dos-test
|
||||
#kafka.output.event.topic.name=DOS-EVENT
|
||||
kafka.output.event.topic.name=abcd
|
||||
|
||||
#kafka输出地址
|
||||
kafka.output.bootstrap.servers=192.168.44.12:9094
|
||||
@@ -38,6 +38,7 @@ kafka.output.bootstrap.servers=192.168.44.12:9094
|
||||
|
||||
#zookeeper地址
|
||||
hbase.zookeeper.quorum=192.168.44.12:2181
|
||||
#hbase.zookeeper.quorum=192.168.40.151:2181,192.168.40.152:2181,192.168.40.203:2181
|
||||
#hbase.zookeeper.quorum=192.168.44.11:2181,192.168.44.14:2181,192.168.44.15:2181
|
||||
|
||||
#hbase客户端处理时间
|
||||
@@ -129,10 +130,68 @@ sasl.jaas.config.password=6MleDyA3Z73HSaXiKsDJ2k7Ys8YWLhEJ
|
||||
sasl.jaas.config.flag=1
|
||||
|
||||
#nacos配置
|
||||
#nacos.server.addr=192.168.44.12:8848
|
||||
#nacos.namespace=public
|
||||
#nacos.username=nacos
|
||||
#nacos.password=nacos
|
||||
#nacos.data.id=knowledge_base.json
|
||||
#nacos.group=DEFAULT_GROUP
|
||||
#nacos.read.timeout=5000
|
||||
|
||||
|
||||
|
||||
############################## Nacos 配置 ######################################
|
||||
nacos.server.addr=192.168.44.12:8848
|
||||
nacos.namespace=public
|
||||
nacos.username=nacos
|
||||
nacos.password=nacos
|
||||
nacos.read.timeout=5000
|
||||
############################## Nacos ---知识库配置 ######################################
|
||||
nacos.namespace=public
|
||||
nacos.data.id=knowledge_base.json
|
||||
nacos.group=DEFAULT_GROUP
|
||||
nacos.read.timeout=5000
|
||||
|
||||
|
||||
############################## Nacos ---静态阈值配置 ######################################
|
||||
nacos.static.namespace=test
|
||||
nacos.static.data.id=dos_detection.properties
|
||||
nacos.static.group=Galaxy
|
||||
|
||||
############################## HTTP 配置 ######################################
|
||||
#http请求相关参数
|
||||
#最大连接数
|
||||
#http.pool.max.connection=400
|
||||
#
|
||||
##单路由最大连接数
|
||||
#http.pool.max.per.route=80
|
||||
#
|
||||
##向服务端请求超时时间设置(单位:毫秒)
|
||||
#http.pool.request.timeout=60000
|
||||
#
|
||||
##向服务端连接超时时间设置(单位:毫秒)
|
||||
#http.pool.connect.timeout=60000
|
||||
#
|
||||
##服务端响应超时时间设置(单位:毫秒)
|
||||
#http.pool.response.timeout=60000
|
||||
|
||||
|
||||
#server.uri=http://192.168.44.12:9098
|
||||
#server.path=/hos/knowledge_base_hos_bucket
|
||||
|
||||
|
||||
|
||||
|
||||
############################## hos Token 配置 ######################################
|
||||
hos.token=c21f969b5f03d33d43e04f8f136e7682
|
||||
|
||||
############################# 选择集群模式或者单机模式 配置 ######################################
|
||||
cluster.or.single=CLUSTER
|
||||
#cluster.or.single=SINGLE
|
||||
|
||||
############################## 集群模式配置文件路径 配置 ######################################
|
||||
hdfs.path=/test/TEST/
|
||||
hdfs.uri.nn1=hdfs://192.168.40.151:9000
|
||||
hdfs.uri.nn2=hdfs://192.168.40.152:9000
|
||||
hdfs.user=dos
|
||||
|
||||
############################## 单机模式配置文件下载路径 配置 ######################################
|
||||
download.path=D:\\ttt\\
|
||||
58
src/main/resources/core-site.xml
Normal file
58
src/main/resources/core-site.xml
Normal file
@@ -0,0 +1,58 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<!--
|
||||
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. See accompanying LICENSE file.
|
||||
-->
|
||||
|
||||
<!-- Put site-specific property overrides in this file. -->
|
||||
|
||||
<configuration>
|
||||
<property>
|
||||
<name>fs.defaultFS</name>
|
||||
<value>hdfs://ns1</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hadoop.tmp.dir</name>
|
||||
<value>file:/home/tsg/olap/hadoop/tmp</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>io.file.buffer.size</name>
|
||||
<value>131702</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hadoop.proxyuser.root.hosts</name>
|
||||
<value>*</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hadoop.proxyuser.root.groups</name>
|
||||
<value>*</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hadoop.logfile.size</name>
|
||||
<value>10000000</value>
|
||||
<description>The max size of each log file</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hadoop.logfile.count</name>
|
||||
<value>1</value>
|
||||
<description>The max number of log files</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ha.zookeeper.quorum</name>
|
||||
<value>192.168.40.151:2181,192.168.40.152:2181,192.168.40.203:2181</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>ipc.client.connect.timeout</name>
|
||||
<value>90000</value>
|
||||
</property>
|
||||
</configuration>
|
||||
142
src/main/resources/hdfs-site.xml
Normal file
142
src/main/resources/hdfs-site.xml
Normal file
@@ -0,0 +1,142 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<!--
|
||||
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. See accompanying LICENSE file.
|
||||
-->
|
||||
|
||||
<!-- Put site-specific property overrides in this file. -->
|
||||
|
||||
<configuration>
|
||||
<property>
|
||||
<name>dfs.namenode.name.dir</name>
|
||||
<value>file:/home/tsg/olap/hadoop/dfs/name</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.datanode.data.dir</name>
|
||||
<value>file:/home/tsg/olap/hadoop/dfs/data</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.replication</name>
|
||||
<value>2</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.webhdfs.enabled</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.permissions</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.permissions.enabled</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.nameservices</name>
|
||||
<value>ns1</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.blocksize</name>
|
||||
<value>134217728</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.ha.namenodes.ns1</name>
|
||||
<value>nn1,nn2</value>
|
||||
</property>
|
||||
<!-- nn1的RPC通信地址,nn1所在地址 -->
|
||||
<property>
|
||||
<name>dfs.namenode.rpc-address.ns1.nn1</name>
|
||||
<value>192.168.40.151:9000</value>
|
||||
</property>
|
||||
<!-- nn1的http通信地址,外部访问地址 -->
|
||||
<property>
|
||||
<name>dfs.namenode.http-address.ns1.nn1</name>
|
||||
<value>192.168.40.151:50070</value>
|
||||
</property>
|
||||
<!-- nn2的RPC通信地址,nn2所在地址 -->
|
||||
<property>
|
||||
<name>dfs.namenode.rpc-address.ns1.nn2</name>
|
||||
<value>192.168.40.152:9000</value>
|
||||
</property>
|
||||
<!-- nn2的http通信地址,外部访问地址 -->
|
||||
<property>
|
||||
<name>dfs.namenode.http-address.ns1.nn2</name>
|
||||
<value>192.168.40.152:50070</value>
|
||||
</property>
|
||||
<!-- 指定NameNode的元数据在JournalNode日志上的存放位置(一般和zookeeper部署在一起) -->
|
||||
<property>
|
||||
<name>dfs.namenode.shared.edits.dir</name>
|
||||
<value>qjournal://192.168.40.151:8485;192.168.40.152:8485;192.168.40.203:8485/ns1</value>
|
||||
</property>
|
||||
<!-- 指定JournalNode在本地磁盘存放数据的位置 -->
|
||||
<property>
|
||||
<name>dfs.journalnode.edits.dir</name>
|
||||
<value>/home/tsg/olap/hadoop/journal</value>
|
||||
</property>
|
||||
<!--客户端通过代理访问namenode,访问文件系统,HDFS 客户端与Active 节点通信的Java 类,使用其确定Active 节点是否活跃 -->
|
||||
<property>
|
||||
<name>dfs.client.failover.proxy.provider.ns1</name>
|
||||
<value>org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider</value>
|
||||
</property>
|
||||
<!--这是配置自动切换的方法,有多种使用方法,具体可以看官网,在文末会给地址,这里是远程登录杀死的方法 -->
|
||||
<property>
|
||||
<name>dfs.ha.fencing.methods</name>
|
||||
<value>sshfence</value>
|
||||
<value>shell(true)</value>
|
||||
</property>
|
||||
<!-- 这个是使用sshfence隔离机制时才需要配置ssh免登陆 -->
|
||||
<property>
|
||||
<name>dfs.ha.fencing.ssh.private-key-files</name>
|
||||
<value>/root/.ssh/id_rsa</value>
|
||||
</property>
|
||||
<!-- 配置sshfence隔离机制超时时间,这个属性同上,如果你是用脚本的方法切换,这个应该是可以不配置的 -->
|
||||
<property>
|
||||
<name>dfs.ha.fencing.ssh.connect-timeout</name>
|
||||
<value>30000</value>
|
||||
</property>
|
||||
<!-- 这个是开启自动故障转移,如果你没有自动故障转移,这个可以先不配 -->
|
||||
<property>
|
||||
<name>dfs.ha.automatic-failover.enabled</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.datanode.max.transfer.threads</name>
|
||||
<value>8192</value>
|
||||
</property>
|
||||
<!-- namenode处理RPC请求线程数,增大该值资源占用不大 -->
|
||||
<property>
|
||||
<name>dfs.namenode.handler.count</name>
|
||||
<value>30</value>
|
||||
</property>
|
||||
<!-- datanode处理RPC请求线程数,增大该值会占用更多内存 -->
|
||||
<property>
|
||||
<name>dfs.datanode.handler.count</name>
|
||||
<value>40</value>
|
||||
</property>
|
||||
<!-- balance时可占用的带宽 -->
|
||||
<property>
|
||||
<name>dfs.balance.bandwidthPerSec</name>
|
||||
<value>104857600</value>
|
||||
</property>
|
||||
<!-- 磁盘预留空间,该空间不会被hdfs占用,单位字节-->
|
||||
<property>
|
||||
<name>dfs.datanode.du.reserved</name>
|
||||
<value>5368709120</value>
|
||||
</property>
|
||||
<!-- datanode与namenode连接超时时间,单位毫秒 2 * heartbeat.recheck.interval + 30000 -->
|
||||
<property>
|
||||
<name>heartbeat.recheck.interval</name>
|
||||
<value>100000</value>
|
||||
</property>
|
||||
</configuration>
|
||||
|
||||
196
src/main/resources/yarn-site.xml
Normal file
196
src/main/resources/yarn-site.xml
Normal file
@@ -0,0 +1,196 @@
|
||||
<?xml version="1.0"?>
|
||||
<!--
|
||||
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. See accompanying LICENSE file.
|
||||
-->
|
||||
<configuration>
|
||||
<property>
|
||||
<name>yarn.nodemanager.aux-services</name>
|
||||
<value>mapreduce_shuffle</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.ha.enabled</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<!--声明两台resourcemanager的地址-->
|
||||
<property>
|
||||
<name>yarn.resourcemanager.cluster-id</name>
|
||||
<value>rmcluster</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.ha.rm-ids</name>
|
||||
<value>rsm1,rsm2</value>
|
||||
</property>
|
||||
<!-- 配置rm1-->
|
||||
<property>
|
||||
<name>yarn.resourcemanager.hostname.rsm1</name>
|
||||
<value>192.168.40.152</value>
|
||||
</property>
|
||||
<property>
|
||||
<!--<name>yarn.resourcemanager.hostname.rm1</name>-->
|
||||
<name>yarn.resourcemanager.address.rsm1</name>
|
||||
<value>192.168.40.152:9916</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.scheduler.address.rsm1</name>
|
||||
<value>192.168.40.152:9917</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.webapp.address.rsm1</name>
|
||||
<value>192.168.40.152:9918</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.admin.address.rsm1</name>
|
||||
<value>192.168.40.152:9919</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.resource-tracker.address.rsm1</name>
|
||||
<value>192.168.40.152:9920</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.ha.admin.address.rsm1</name>
|
||||
<value>192.168.40.152:23142</value>
|
||||
</property>
|
||||
<!-- 配置rm2-->
|
||||
|
||||
<property>
|
||||
<name>yarn.resourcemanager.hostname.rsm2</name>
|
||||
<value>192.168.40.203</value>
|
||||
</property>
|
||||
<property>
|
||||
<!--<name>yarn.resourcemanager.hostname.rm1</name>-->
|
||||
<name>yarn.resourcemanager.address.rsm2</name>
|
||||
<value>192.168.40.203:9916</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.scheduler.address.rsm2</name>
|
||||
<value>192.168.40.203:9917</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.webapp.address.rsm2</name>
|
||||
<value>192.168.40.203:9918</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.admin.address.rsm2</name>
|
||||
<value>192.168.40.203:9919</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.resource-tracker.address.rsm2</name>
|
||||
<value>192.168.40.203:9920</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.ha.admin.address.rsm2</name>
|
||||
<value>192.168.40.203:23142</value>
|
||||
</property>
|
||||
<!--指定zookeeper集群的地址-->
|
||||
<property>
|
||||
<name>yarn.resourcemanager.zk-address</name>
|
||||
<value>192.168.40.151:2181,192.168.40.152:2181,192.168.40.203:2181</value>
|
||||
</property>
|
||||
<!--启用自动恢复,当任务进行一半,rm坏掉,就要启动自动恢复,默认是false-->
|
||||
<property>
|
||||
<name>yarn.resourcemanager.recovery.enabled</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<!--启动nm自动恢复,当集群重启,container开启自动恢复,保障任务的可靠性,默认为false,启动该配置需开启下面配置 -->
|
||||
<property>
|
||||
<name>yarn.nodemanager.recovery.enabled</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<!--开启nm故障恢复后,nm元数据存储路径 -->
|
||||
<property>
|
||||
<name>yarn.nodemanager.recovery.dir</name>
|
||||
<value>/home/tsg/olap/hadoop-2.7.1/yarn</value>
|
||||
</property>
|
||||
<!--启用nm恢复时监控功能,开启后不会去尝试清理container,默认false -->
|
||||
<property>
|
||||
<name>yarn.nodemanager.recovery.supervised</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<!--配置nm可用的RPC地址,默认${yarn.nodemanager.hostname}:0,为临时端口。集群重启后,nm与rm连接的端口会变化,这里指定端口,保障nm restart功能 -->
|
||||
<property>
|
||||
<name>yarn.nodemanager.address</name>
|
||||
<value>${yarn.nodemanager.hostname}:9923</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.resourcemanager.store.class</name>
|
||||
<value>org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.nodemanager.resource.memory-mb</name>
|
||||
<value>30720</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.scheduler.minimum-allocation-mb</name>
|
||||
<value>1024</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>yarn.scheduler.maximum-allocation-mb</name>
|
||||
<value>30720</value>
|
||||
</property>
|
||||
<!--开启日志聚合 -->
|
||||
<property>
|
||||
<name>yarn.log-aggregation-enable</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.nodemanager.heartbeat-interval-ms</name>
|
||||
<value>3000</value>
|
||||
</property>
|
||||
<!--日志保留7天 -->
|
||||
<property>
|
||||
<name>yarn.log-aggregation.retain-seconds</name>
|
||||
<value>604800</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds</name>
|
||||
<value>3600</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.nodemanager.remote-app-log-dir</name>
|
||||
<value>/tmp/logs</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.nodemanager.resource.cpu-vcores</name>
|
||||
<value>14</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.scheduler.minimum-allocation-vcores</name>
|
||||
<value>1</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.scheduler.maximum-allocation-vcores</name>
|
||||
<value>14</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.nodemanager.vmem-check-enabled</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.nodemanager.pmem-check-enabled</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.nodemanager.disk-health-checker.enable</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
<!--ApplicationMaster重启次数,配置HA后默认为2-->
|
||||
<property>
|
||||
<name>yarn.resourcemanager.am.max-attempts</name>
|
||||
<value>10000</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>yarn.log.server.url</name>
|
||||
<value>http://bigdata-151:19888/jobhistory/logs</value>
|
||||
</property>
|
||||
</configuration>
|
||||
Reference in New Issue
Block a user