diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/InstanceProfile.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/InstanceProfile.java index 9e85872ff54..c3b20a05818 100644 --- a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/InstanceProfile.java +++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/InstanceProfile.java @@ -20,6 +20,7 @@ import org.apache.inlong.agent.constant.TaskConstants; import org.apache.inlong.agent.utils.file.FileUtils; import org.apache.inlong.common.enums.InstanceStateEnum; +import org.apache.inlong.common.enums.TaskTypeEnum; import org.apache.inlong.common.pojo.dataproxy.DataProxyTopicInfo; import org.apache.inlong.common.pojo.dataproxy.MQClusterInfo; @@ -40,12 +41,24 @@ import static org.apache.inlong.agent.constant.TaskConstants.TASK_MQ_CLUSTERS; import static org.apache.inlong.agent.constant.TaskConstants.TASK_MQ_TOPIC; import static org.apache.inlong.agent.constant.TaskConstants.TASK_RETRY; +import static org.apache.inlong.agent.constant.TaskConstants.TASK_TYPE; /** * job profile which contains details describing properties of one job. */ public class InstanceProfile extends AbstractConfiguration implements Comparable { + public static final String DEFAULT_FILE_INSTANCE = "org.apache.inlong.agent.plugin.instance.FileInstance"; + public static final String DEFAULT_COS_INSTANCE = "org.apache.inlong.agent.plugin.instance.COSInstance"; + public static final String DEFAULT_KAFKA_INSTANCE = "org.apache.inlong.agent.plugin.instance.KafkaInstance"; + public static final String DEFAULT_MONGODB_INSTANCE = "org.apache.inlong.agent.plugin.instance.MongoDBInstance"; + public static final String DEFAULT_MQTT_INSTANCE = "org.apache.inlong.agent.plugin.instance.MqttInstance"; + public static final String DEFAULT_ORACLE_INSTANCE = "org.apache.inlong.agent.plugin.instance.OracleInstance"; + public static final String DEFAULT_POSTGRES_INSTANCE = "org.apache.inlong.agent.plugin.instance.PostgreSQLInstance"; + public static final String DEFAULT_PULSAR_INSTANCE = "org.apache.inlong.agent.plugin.instance.PulsarInstance"; + public static final String DEFAULT_REDIS_INSTANCE = "org.apache.inlong.agent.plugin.instance.RedisInstance"; + public static final String DEFAULT_SQLSERVER_INSTANCE = "org.apache.inlong.agent.plugin.instance.SQLServerInstance"; + private static final Logger LOGGER = LoggerFactory.getLogger(InstanceProfile.class); private static final Gson GSON = new Gson(); @@ -64,12 +77,40 @@ public String toJsonStr() { return GSON.toJson(getConfigStorage()); } - public void setInstanceClass(String className) { - set(TaskConstants.INSTANCE_CLASS, className); + public String getInstanceClass() { + TaskTypeEnum taskType = TaskTypeEnum.getTaskType(getInt(TASK_TYPE, TaskTypeEnum.FILE.getType())); + return getInstanceClassByTaskType(taskType); } - public String getInstanceClass() { - return get(TaskConstants.INSTANCE_CLASS); + public static String getInstanceClassByTaskType(TaskTypeEnum taskType) { + if (taskType == null) { + return null; + } + switch (taskType) { + case FILE: + return DEFAULT_FILE_INSTANCE; + case KAFKA: + return DEFAULT_KAFKA_INSTANCE; + case PULSAR: + return DEFAULT_PULSAR_INSTANCE; + case POSTGRES: + return DEFAULT_POSTGRES_INSTANCE; + case ORACLE: + return DEFAULT_ORACLE_INSTANCE; + case SQLSERVER: + return DEFAULT_SQLSERVER_INSTANCE; + case MONGODB: + return DEFAULT_MONGODB_INSTANCE; + case REDIS: + return DEFAULT_REDIS_INSTANCE; + case MQTT: + return DEFAULT_MQTT_INSTANCE; + case COS: + return DEFAULT_COS_INSTANCE; + default: + LOGGER.error("invalid task type {}", taskType); + return null; + } } public String getTaskId() { diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/TaskProfile.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/TaskProfile.java index 1f77433c9f4..c2a60a0598f 100644 --- a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/TaskProfile.java +++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/conf/TaskProfile.java @@ -23,6 +23,7 @@ import org.apache.inlong.agent.utils.DateTransUtils; import org.apache.inlong.common.enums.InstanceStateEnum; import org.apache.inlong.common.enums.TaskStateEnum; +import org.apache.inlong.common.enums.TaskTypeEnum; import org.apache.inlong.common.pojo.agent.DataConfig; import com.google.gson.Gson; @@ -32,18 +33,32 @@ import java.text.ParseException; import java.util.TimeZone; +import static java.util.Objects.requireNonNull; import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_GROUP_ID; import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_INLONG_STREAM_ID; import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_GROUP_ID; import static org.apache.inlong.agent.constant.CommonConstants.PROXY_INLONG_STREAM_ID; import static org.apache.inlong.agent.constant.TaskConstants.TASK_RETRY; import static org.apache.inlong.agent.constant.TaskConstants.TASK_STATE; +import static org.apache.inlong.agent.constant.TaskConstants.TASK_TYPE; /** * job profile which contains details describing properties of one job. */ public class TaskProfile extends AbstractConfiguration { + public static final String DEFAULT_FILE_TASK = "org.apache.inlong.agent.plugin.task.logcollection.local.FileTask"; + public static final String DEFAULT_COS_TASK = "org.apache.inlong.agent.plugin.task.logcollection.cos.COSTask"; + public static final String DEFAULT_KAFKA_TASK = "org.apache.inlong.agent.plugin.task.KafkaTask"; + public static final String DEFAULT_PULSAR_TASK = "org.apache.inlong.agent.plugin.task.PulsarTask"; + public static final String DEFAULT_MONGODB_TASK = "org.apache.inlong.agent.plugin.task.MongoDBTask"; + public static final String DEFAULT_ORACLE_TASK = "org.apache.inlong.agent.plugin.task.OracleTask"; + public static final String DEFAULT_REDIS_TASK = "org.apache.inlong.agent.plugin.task.RedisTask"; + public static final String DEFAULT_POSTGRESQL_TASK = "org.apache.inlong.agent.plugin.task.PostgreSQLTask"; + public static final String DEFAULT_MQTT_TASK = "org.apache.inlong.agent.plugin.task.MqttTask"; + public static final String DEFAULT_SQLSERVER_TASK = "org.apache.inlong.agent.plugin.task.SQLServerTask"; + public static final String DEFAULT_MOCK_TASK = "org.apache.inlong.agent.plugin.task.MockTask"; + private static final Gson GSON = new Gson(); private static final Logger logger = LoggerFactory.getLogger(TaskProfile.class); @@ -57,6 +72,37 @@ public static TaskProfile convertToTaskProfile(DataConfig dataConfig) { return TaskProfileDto.convertToTaskProfile(dataConfig); } + public String getTaskClass() { + TaskTypeEnum taskType = TaskTypeEnum.getTaskType(getInt(TASK_TYPE, TaskTypeEnum.FILE.getType())); + switch (requireNonNull(taskType)) { + case FILE: + return DEFAULT_FILE_TASK; + case KAFKA: + return DEFAULT_KAFKA_TASK; + case PULSAR: + return DEFAULT_PULSAR_TASK; + case POSTGRES: + return DEFAULT_POSTGRESQL_TASK; + case ORACLE: + return DEFAULT_ORACLE_TASK; + case SQLSERVER: + return DEFAULT_SQLSERVER_TASK; + case MONGODB: + return DEFAULT_MONGODB_TASK; + case REDIS: + return DEFAULT_REDIS_TASK; + case MQTT: + return DEFAULT_MQTT_TASK; + case COS: + return DEFAULT_COS_TASK; + case MOCK: + return DEFAULT_MOCK_TASK; + default: + logger.error("invalid task type {}", taskType); + return null; + } + } + public String getTaskId() { return get(TaskConstants.TASK_ID); } @@ -65,10 +111,6 @@ public String getCycleUnit() { return get(TaskConstants.TASK_CYCLE_UNIT); } - public String getTimeOffset() { - return get(TaskConstants.TASK_FILE_TIME_OFFSET, ""); - } - public String getTimeZone() { return get(TaskConstants.TASK_TIME_ZONE); } @@ -85,14 +127,6 @@ public boolean isRetry() { return getBoolean(TASK_RETRY, false); } - public String getTaskClass() { - return get(TaskConstants.TASK_CLASS); - } - - public void setTaskClass(String className) { - set(TaskConstants.TASK_CLASS, className); - } - public String getInlongGroupId() { return get(PROXY_INLONG_GROUP_ID, DEFAULT_PROXY_INLONG_GROUP_ID); } @@ -128,11 +162,9 @@ public String toJsonStr() { return GSON.toJson(getConfigStorage()); } - public InstanceProfile createInstanceProfile(String instanceClass, String fileName, String cycleUnit, - String dataTime, + public InstanceProfile createInstanceProfile(String fileName, String cycleUnit, String dataTime, long fileUpdateTime) { InstanceProfile instanceProfile = InstanceProfile.parseJsonStr(toJsonStr()); - instanceProfile.setInstanceClass(instanceClass); instanceProfile.setInstanceId(fileName); instanceProfile.setSourceDataTime(dataTime); Long sinkDataTime = 0L; diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constant/TaskConstants.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constant/TaskConstants.java index 5dbc353d254..201ec247136 100755 --- a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constant/TaskConstants.java +++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/constant/TaskConstants.java @@ -22,11 +22,8 @@ */ public class TaskConstants extends CommonConstants { - // job id - // public static final String JOB_ID = "job.id"; public static final String TASK_ID = "task.id"; public static final String INSTANCE_ID = "instance.id"; - public static final String JOB_INSTANCE_ID = "job.instance.id"; public static final String INSTANCE_CREATE_TIME = "instance.createTime"; public static final String INSTANCE_MODIFY_TIME = "instance.modifyTime"; public static final String TASK_GROUP_ID = "task.groupId"; @@ -36,9 +33,7 @@ public class TaskConstants extends CommonConstants { public static final String TASK_SOURCE = "task.source"; public static final String TASK_CHANNEL = "task.channel"; - - public static final String TASK_CLASS = "task.taskClass"; - public static final String INSTANCE_CLASS = "task.instance.class"; + public static final String TASK_TYPE = "task.taskType"; public static final String TASK_FILE_TRIGGER = "task.fileTask.trigger"; // sink config @@ -59,12 +54,12 @@ public class TaskConstants extends CommonConstants { public static final String TASK_CYCLE_UNIT = "task.cycleUnit"; public static final String FILE_TASK_CYCLE_UNIT = "task.fileTask.cycleUnit"; public static final String TASK_FILE_CONTENT_COLLECT_TYPE = "task.fileTask.contentCollectType"; - public static final String SOURCE_DATA_CONTENT_STYLE = "task.fileTask.dataContentStyle"; - public static final String SOURCE_DATA_SEPARATOR = "task.fileTask.dataSeparator"; - public static final String SOURCE_FILTER_STREAMS = "task.fileTask.filterStreams"; - public static final String TASK_RETRY = "task.fileTask.retry"; - public static final String TASK_START_TIME = "task.fileTask.startTime"; - public static final String TASK_END_TIME = "task.fileTask.endTime"; + public static final String FILE_CONTENT_STYLE = "task.fileTask.dataContentStyle"; + public static final String FILE_DATA_SEPARATOR = "task.fileTask.dataSeparator"; + public static final String FILE_FILTER_STREAMS = "task.fileTask.filterStreams"; + public static final String TASK_RETRY = "task.retry"; + public static final String FILE_TASK_TIME_FROM = "task.fileTask.dataTimeFrom"; + public static final String FILE_TASK_TIME_TO = "task.fileTask.dataTimeTo"; public static final String FILE_MAX_NUM = "task.fileTask.maxFileCount"; public static final String PREDEFINE_FIELDS = "task.predefinedFields"; public static final String TASK_AUDIT_VERSION = "task.auditVersion"; @@ -75,6 +70,22 @@ public class TaskConstants extends CommonConstants { public static final String TASK_KAFKA_OFFSET = "task.kafkaTask.partition.offset"; public static final String TASK_KAFKA_AUTO_COMMIT_OFFSET_RESET = "task.kafkaTask.autoOffsetReset"; + // COS task + public static final String COS_TASK_CYCLE_UNIT = "task.cosTask.cycleUnit"; + public static final String COS_CONTENT_STYLE = "task.cosTask.contentStyle"; + public static final String COS_MAX_NUM = "task.cosTask.maxFileCount"; + public static final String COS_TASK_PATTERN = "task.cosTask.pattern"; + public static final String TASK_COS_TIME_OFFSET = "task.cosTask.timeOffset"; + public static final String COS_TASK_RETRY = "task.cosTask.retry"; + public static final String COS_TASK_TIME_FROM = "task.cosTask.dataTimeFrom"; + public static final String COS_TASK_TIME_TO = "task.cosTask.dataTimeTo"; + public static final String COS_TASK_BUCKET_NAME = "task.cosTask.bucketName"; + public static final String COS_TASK_SECRET_ID = "task.cosTask.secretId"; + public static final String COS_TASK_SECRET_KEY = "task.cosTask.secretKey"; + public static final String COS_TASK_REGION = "task.cosTask.region"; + public static final String COS_DATA_SEPARATOR = "task.cosTask.dataSeparator"; + public static final String COS_FILTER_STREAMS = "task.cosTask.filterStreams"; + /** * delimiter to split offset for different task */ diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/audit/AuditUtils.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/audit/AuditUtils.java index b4f74bfc63c..76e4b8f31e4 100644 --- a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/audit/AuditUtils.java +++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/metrics/audit/AuditUtils.java @@ -22,6 +22,9 @@ import org.apache.inlong.audit.AuditOperator; import org.apache.inlong.audit.entity.AuditComponent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.HashSet; import static org.apache.inlong.agent.constant.AgentConstants.AUDIT_ENABLE; @@ -38,6 +41,8 @@ */ public class AuditUtils { + private static final Logger LOGGER = LoggerFactory.getLogger(AuditUtils.class); + public static final int AGENT_ISOLATE_KEY = 1; public static int AUDIT_ID_AGENT_READ_SUCCESS = 3; public static int AUDIT_ID_AGENT_SEND_SUCCESS = 4; public static int AUDIT_ID_AGENT_READ_FAILED = 524291; @@ -90,8 +95,17 @@ public static void add(int auditID, String inlongGroupId, String inlongStreamId, if (!IS_AUDIT) { return; } - AuditOperator.getInstance() - .add(auditID, DEFAULT_AUDIT_TAG, inlongGroupId, inlongStreamId, logTime, count, size, version); + if (inlongGroupId == null || inlongStreamId == null) { + LOGGER.error("invalid args inlongGroupId: {}, inlongStreamId: {}", inlongGroupId, inlongStreamId); + return; + } + try { + AuditOperator.getInstance() + .add(auditID, DEFAULT_AUDIT_TAG, inlongGroupId, inlongStreamId, logTime, count, size, version); + } catch (Throwable e) { + LOGGER.error("call audit add inlongGroupId: {}, inlongStreamId: {}, auditID {}, error", inlongGroupId, + inlongStreamId, auditID, e); + } } public static void add(int auditID, String inlongGroupId, String inlongStreamId, @@ -106,6 +120,6 @@ public static void send() { if (!IS_AUDIT) { return; } - AuditOperator.getInstance().flush(); + AuditOperator.getInstance().flush(AGENT_ISOLATE_KEY); } } diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Instance.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Instance.java index 0d43587f6eb..e67d5438821 100755 --- a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Instance.java +++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/plugin/Instance.java @@ -39,6 +39,11 @@ public abstract class Instance extends AbstractStateWrapper { */ public abstract void destroy(); + /** + * notify destroy instance. + */ + public abstract void notifyDestroy(); + /** * get instance profile */ diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/COSTask.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/COSTask.java new file mode 100644 index 00000000000..83d43c6af3d --- /dev/null +++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/COSTask.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.agent.pojo; + +import lombok.Data; + +import java.util.List; + +@Data +public class COSTask { + + private Integer id; + private String pattern; + private String cycleUnit; + private Boolean retry; + private String dataTimeFrom; + private String dataTimeTo; + private String timeOffset; + private Integer maxFileCount; + private String collectType; + private String contentStyle; + private String dataSeparator; + private String filterStreams; + private String bucketName; + private String secretId; + private String secretKey; + private String region; + + @Data + public static class COSTaskConfig { + + private String pattern; + private String cycleUnit; + private Boolean retry; + private String dataTimeFrom; + private String dataTimeTo; + // '1m' means one minute after, '-1m' means one minute before + // '1h' means one hour after, '-1h' means one hour before + // '1d' means one day after, '-1d' means one day before + // Null means from current timestamp + private String timeOffset; + private Integer maxFileCount; + // Collect type, for example: FULL, INCREMENT + private String collectType; + // Type of data result for column separator + // CSV format, set this parameter to a custom separator: , | : + // Json format, set this parameter to json + private String contentStyle; + // Column separator of data source + private String dataSeparator; + // The streamIds to be filtered out + private List filterStreams; + private String bucketName; + private String credentialsId; + private String credentialsKey; + private String region; + } +} diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/FileTask.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/FileTask.java index 57c294f7d41..54c191ffcaa 100644 --- a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/FileTask.java +++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/FileTask.java @@ -29,8 +29,8 @@ public class FileTask { private Integer id; private String cycleUnit; private Boolean retry; - private Long startTime; - private Long endTime; + private String dataTimeFrom; + private String dataTimeTo; private String timeOffset; private String timeZone; private String addictiveString; @@ -91,9 +91,9 @@ public static class FileTaskConfig { private Boolean retry; - private Long startTime; + private String dataTimeFrom; - private Long endTime; + private String dataTimeTo; private String pattern; diff --git a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/TaskProfileDto.java b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/TaskProfileDto.java index 85c636c8851..a30bf44ecc0 100644 --- a/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/TaskProfileDto.java +++ b/inlong-agent/agent-common/src/main/java/org/apache/inlong/agent/pojo/TaskProfileDto.java @@ -21,6 +21,7 @@ import org.apache.inlong.agent.conf.TaskProfile; import org.apache.inlong.agent.constant.CycleUnitType; import org.apache.inlong.agent.pojo.BinlogTask.BinlogTaskConfig; +import org.apache.inlong.agent.pojo.COSTask.COSTaskConfig; import org.apache.inlong.agent.pojo.FileTask.FileTaskConfig; import org.apache.inlong.agent.pojo.FileTask.Line; import org.apache.inlong.agent.pojo.KafkaTask.KafkaTaskConfig; @@ -37,6 +38,8 @@ import com.google.gson.Gson; import lombok.Data; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.stream.Collectors; @@ -48,21 +51,12 @@ @Data public class TaskProfileDto { - public static final String DEFAULT_FILE_TASK = "org.apache.inlong.agent.plugin.task.file.LogFileTask"; - public static final String DEFAULT_KAFKA_TASK = "org.apache.inlong.agent.plugin.task.KafkaTask"; - public static final String DEFAULT_PULSAR_TASK = "org.apache.inlong.agent.plugin.task.PulsarTask"; - public static final String DEFAULT_MONGODB_TASK = "org.apache.inlong.agent.plugin.task.MongoDBTask"; - public static final String DEFAULT_ORACLE_TASK = "org.apache.inlong.agent.plugin.task.OracleTask"; - public static final String DEFAULT_REDIS_TASK = "org.apache.inlong.agent.plugin.task.RedisTask"; - public static final String DEFAULT_POSTGRESQL_TASK = "org.apache.inlong.agent.plugin.task.PostgreSQLTask"; - public static final String DEFAULT_MQTT_TASK = "org.apache.inlong.agent.plugin.task.MqttTask"; - public static final String DEFAULT_SQLSERVER_TASK = "org.apache.inlong.agent.plugin.task.SQLServerTask"; + private static final Logger logger = LoggerFactory.getLogger(TaskProfileDto.class); public static final String DEFAULT_CHANNEL = "org.apache.inlong.agent.plugin.channel.MemoryChannel"; public static final String MANAGER_JOB = "MANAGER_JOB"; public static final String DEFAULT_DATA_PROXY_SINK = "org.apache.inlong.agent.plugin.sinks.ProxySink"; public static final String PULSAR_SINK = "org.apache.inlong.agent.plugin.sinks.PulsarSink"; public static final String KAFKA_SINK = "org.apache.inlong.agent.plugin.sinks.KafkaSink"; - /** * file source */ @@ -101,6 +95,10 @@ public class TaskProfileDto { * sqlserver source */ public static final String SQLSERVER_SOURCE = "org.apache.inlong.agent.plugin.sources.SQLServerSource"; + /** + * cos source + */ + public static final String COS_SOURCE = "org.apache.inlong.agent.plugin.sources.COSSource"; private static final Gson GSON = new Gson(); @@ -166,8 +164,8 @@ private static FileTask getFileTask(DataConfig dataConfig) { fileTask.setMaxFileCount(taskConfig.getMaxFileCount()); fileTask.setRetry(taskConfig.getRetry()); fileTask.setCycleUnit(taskConfig.getCycleUnit()); - fileTask.setStartTime(taskConfig.getStartTime()); - fileTask.setEndTime(taskConfig.getEndTime()); + fileTask.setDataTimeFrom(taskConfig.getDataTimeFrom()); + fileTask.setDataTimeTo(taskConfig.getDataTimeTo()); if (taskConfig.getFilterStreams() != null) { fileTask.setFilterStreams(GSON.toJson(taskConfig.getFilterStreams())); } @@ -197,6 +195,35 @@ private static FileTask getFileTask(DataConfig dataConfig) { return fileTask; } + private static COSTask getCOSTask(DataConfig dataConfig) { + COSTask cosTask = new COSTask(); + cosTask.setId(dataConfig.getTaskId()); + COSTaskConfig taskConfig = GSON.fromJson(dataConfig.getExtParams(), + COSTaskConfig.class); + cosTask.setPattern(taskConfig.getPattern()); + cosTask.setCollectType(taskConfig.getCollectType()); + cosTask.setContentStyle(taskConfig.getContentStyle()); + cosTask.setDataSeparator(taskConfig.getDataSeparator()); + cosTask.setMaxFileCount(taskConfig.getMaxFileCount()); + cosTask.setRetry(taskConfig.getRetry()); + cosTask.setCycleUnit(taskConfig.getCycleUnit()); + cosTask.setDataTimeFrom(taskConfig.getDataTimeFrom()); + cosTask.setDataTimeTo(taskConfig.getDataTimeTo()); + cosTask.setBucketName(taskConfig.getBucketName()); + cosTask.setSecretId(taskConfig.getCredentialsId()); + cosTask.setSecretKey(taskConfig.getCredentialsKey()); + cosTask.setRegion(taskConfig.getRegion()); + if (taskConfig.getFilterStreams() != null) { + cosTask.setFilterStreams(GSON.toJson(taskConfig.getFilterStreams())); + } + if (taskConfig.getTimeOffset() != null) { + cosTask.setTimeOffset(taskConfig.getTimeOffset()); + } else { + cosTask.setTimeOffset(deafult_time_offset + cosTask.getCycleUnit()); + } + return cosTask; + } + private static KafkaTask getKafkaTask(DataConfig dataConfigs) { KafkaTaskConfig kafkaTaskConfig = GSON.fromJson(dataConfigs.getExtParams(), @@ -432,6 +459,7 @@ public static TaskProfile convertToTaskProfile(DataConfig dataConfig) { // common attribute task.setId(String.valueOf(dataConfig.getTaskId())); + task.setTaskType(dataConfig.getTaskType()); task.setGroupId(dataConfig.getInlongGroupId()); task.setStreamId(dataConfig.getInlongStreamId()); task.setChannel(DEFAULT_CHANNEL); @@ -468,6 +496,7 @@ public static TaskProfile convertToTaskProfile(DataConfig dataConfig) { throw new IllegalArgumentException("invalid mq type " + mqType + " please check"); } } + task.setRetry(false); TaskTypeEnum taskType = TaskTypeEnum.getTaskType(dataConfig.getTaskType()); switch (requireNonNull(taskType)) { case SQL: @@ -478,64 +507,56 @@ public static TaskProfile convertToTaskProfile(DataConfig dataConfig) { profileDto.setTask(task); break; case FILE: - task.setTaskClass(DEFAULT_FILE_TASK); FileTask fileTask = getFileTask(dataConfig); task.setCycleUnit(fileTask.getCycleUnit()); task.setFileTask(fileTask); task.setSource(DEFAULT_SOURCE); + task.setRetry(fileTask.getRetry()); profileDto.setTask(task); break; case KAFKA: - task.setTaskClass(DEFAULT_KAFKA_TASK); KafkaTask kafkaTask = getKafkaTask(dataConfig); task.setKafkaTask(kafkaTask); task.setSource(KAFKA_SOURCE); profileDto.setTask(task); break; case PULSAR: - task.setTaskClass(DEFAULT_PULSAR_TASK); PulsarTask pulsarTask = getPulsarTask(dataConfig); task.setPulsarTask(pulsarTask); task.setSource(PULSAR_SOURCE); profileDto.setTask(task); break; case POSTGRES: - task.setTaskClass(DEFAULT_POSTGRESQL_TASK); PostgreSQLTask postgreSQLTask = getPostgresTask(dataConfig); task.setPostgreSQLTask(postgreSQLTask); task.setSource(POSTGRESQL_SOURCE); profileDto.setTask(task); break; case ORACLE: - task.setTaskClass(DEFAULT_ORACLE_TASK); OracleTask oracleTask = getOracleTask(dataConfig); task.setOracleTask(oracleTask); task.setSource(ORACLE_SOURCE); profileDto.setTask(task); break; case SQLSERVER: - task.setTaskClass(DEFAULT_SQLSERVER_TASK); SqlServerTask sqlserverTask = getSqlServerTask(dataConfig); task.setSqlserverTask(sqlserverTask); task.setSource(SQLSERVER_SOURCE); profileDto.setTask(task); break; case MONGODB: - task.setTaskClass(DEFAULT_MONGODB_TASK); MongoTask mongoTask = getMongoTask(dataConfig); task.setMongoTask(mongoTask); task.setSource(MONGO_SOURCE); profileDto.setTask(task); break; case REDIS: - task.setTaskClass(DEFAULT_REDIS_TASK); RedisTask redisTask = getRedisTask(dataConfig); task.setRedisTask(redisTask); task.setSource(REDIS_SOURCE); profileDto.setTask(task); break; case MQTT: - task.setTaskClass(DEFAULT_MQTT_TASK); MqttTask mqttTask = getMqttTask(dataConfig); task.setMqttTask(mqttTask); task.setSource(MQTT_SOURCE); @@ -544,7 +565,16 @@ public static TaskProfile convertToTaskProfile(DataConfig dataConfig) { case MOCK: profileDto.setTask(task); break; + case COS: + COSTask cosTask = getCOSTask(dataConfig); + task.setCycleUnit(cosTask.getCycleUnit()); + task.setCosTask(cosTask); + task.setSource(COS_SOURCE); + task.setRetry(cosTask.getRetry()); + profileDto.setTask(task); + break; default: + logger.error("invalid task type {}", taskType); } return TaskProfile.parseJsonStr(GSON.toJson(profileDto)); } @@ -569,11 +599,13 @@ public static class Task { private String mqClusters; private String topicInfo; private String taskClass; + private Integer taskType; private String predefinedFields; private Integer state; private String cycleUnit; private String timeZone; private String auditVersion; + private boolean retry; private FileTask fileTask; private BinlogTask binlogTask; @@ -585,6 +617,7 @@ public static class Task { private RedisTask redisTask; private MqttTask mqttTask; private SqlServerTask sqlserverTask; + private COSTask cosTask; } @Data diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/instance/InstanceManager.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/instance/InstanceManager.java index 06dd20a99e5..3396c3a5912 100644 --- a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/instance/InstanceManager.java +++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/instance/InstanceManager.java @@ -65,6 +65,7 @@ public class InstanceManager extends AbstractDaemon { private final ConcurrentHashMap instanceMap; // instance profile queue. private final BlockingQueue actionQueue; + private final BlockingQueue addActionQueue; // task thread pool; private static final ThreadPoolExecutor EXECUTOR_SERVICE = new ThreadPoolExecutor( 0, Integer.MAX_VALUE, @@ -125,6 +126,7 @@ public InstanceManager(String taskId, int instanceLimit, Store basicStore, TaskS instanceMap = new ConcurrentHashMap<>(); this.instanceLimit = instanceLimit; actionQueue = new LinkedBlockingQueue<>(ACTION_QUEUE_CAPACITY); + addActionQueue = new LinkedBlockingQueue<>(ACTION_QUEUE_CAPACITY); } public String getTaskId() { @@ -147,6 +149,9 @@ public boolean submitAction(InstanceAction action) { if (action == null) { return false; } + if (isFull()) { + return false; + } return actionQueue.offer(action); } @@ -163,7 +168,8 @@ private Runnable coreThread() { try { AgentUtils.silenceSleepInMs(CORE_THREAD_SLEEP_TIME_MS); printInstanceState(); - dealWithActionQueue(actionQueue); + dealWithActionQueue(); + dealWithAddActionQueue(); keepPaceWithStore(); String inlongGroupId = taskFromStore.getInlongGroupId(); String inlongStreamId = taskFromStore.getInlongStreamId(); @@ -251,16 +257,18 @@ private void traverseMemoryTasksToStore() { }); } - private void dealWithActionQueue(BlockingQueue queue) { + private void dealWithActionQueue() { while (isRunnable()) { try { - InstanceAction action = queue.poll(); + InstanceAction action = actionQueue.poll(); if (action == null) { break; } switch (action.getActionType()) { case ADD: - addInstance(action.getProfile()); + if (!addActionQueue.offer(action)) { + LOGGER.error("it should never happen: addQueue is full"); + } break; case FINISH: finishInstance(action.getProfile()); @@ -279,6 +287,20 @@ private void dealWithActionQueue(BlockingQueue queue) { } } + private void dealWithAddActionQueue() { + while (isRunnable()) { + if (instanceMap.size() > instanceLimit) { + LOGGER.error("instanceMap size {} over limit {}", instanceMap.size(), instanceLimit); + return; + } + InstanceAction action = addActionQueue.poll(); + if (action == null) { + break; + } + addInstance(action.getProfile()); + } + } + @Override public void start() { restoreFromStore(); @@ -317,12 +339,6 @@ private void restoreFromStore() { } private void addInstance(InstanceProfile profile) { - if (instanceMap.size() > instanceLimit) { - LOGGER.error("instanceMap size {} over limit {}", instanceMap.size(), instanceLimit); - actionQueue.offer(new InstanceAction(ActionType.ADD, profile)); - AgentUtils.silenceSleepInMs(CORE_THREAD_SLEEP_TIME_MS); - return; - } LOGGER.info("add instance taskId {} instanceId {}", taskId, profile.getInstanceId()); if (!shouldAddAgain(profile.getInstanceId(), profile.getFileUpdateTime())) { LOGGER.info("addInstance shouldAddAgain returns false skip taskId {} instanceId {}", taskId, @@ -375,6 +391,15 @@ private void deleteFromMemory(String instanceId) { instance.getProfile().getSinkDataTime(), 1, 1, auditVersion); } + private void notifyDestroyInstance(String instanceId) { + Instance instance = instanceMap.get(instanceId); + if (instance == null) { + LOGGER.error("try to notify destroy instance but not found: taskId {} instanceId {}", taskId, instanceId); + return; + } + instance.notifyDestroy(); + } + private void addToStore(InstanceProfile profile, boolean addNew) { LOGGER.info("add instance to instance store state {} instanceId {}", profile.getState(), profile.getInstanceId()); @@ -428,11 +453,14 @@ private void addToMemory(InstanceProfile instanceProfile) { instanceProfile.getSinkDataTime(), 1, 1, auditVersion); } } catch (Throwable t) { - LOGGER.error("add instance error {}", t.getMessage()); + LOGGER.error("add instance error.", t); } } private void stopAllInstances() { + instanceMap.values().forEach((instance) -> { + notifyDestroyInstance(instance.getInstanceId()); + }); instanceMap.values().forEach((instance) -> { deleteFromMemory(instance.getInstanceId()); }); @@ -459,7 +487,7 @@ public boolean shouldAddAgain(String fileName, long lastModifyTime) { } public boolean isFull() { - return (instanceMap.size() + actionQueue.size()) >= instanceLimit * reserveCoefficient; + return (actionQueue.size() + addActionQueue.size()) >= ACTION_QUEUE_CAPACITY * reserveCoefficient; } public long getFinishedInstanceCount() { diff --git a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/MemoryManager.java b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/MemoryManager.java index d785effdac8..b262e171e92 100644 --- a/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/MemoryManager.java +++ b/inlong-agent/agent-core/src/main/java/org/apache/inlong/agent/core/task/MemoryManager.java @@ -23,6 +23,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashSet; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Semaphore; @@ -44,6 +46,7 @@ public class MemoryManager { private ConcurrentHashMap semaphoreMap = new ConcurrentHashMap<>(); private ConcurrentHashMap lastPrintTime = new ConcurrentHashMap<>(); private static final int PRINT_INTERVAL_MS = 1000; + private Set defaultSemaphoreTypes = new HashSet<>(); private MemoryManager() { this.conf = AgentConfiguration.getAgentConf(); @@ -52,16 +55,19 @@ private MemoryManager() { conf.getInt(AGENT_GLOBAL_READER_SOURCE_PERMIT, DEFAULT_AGENT_GLOBAL_READER_SOURCE_PERMIT)); semaphoreMap.put(AGENT_GLOBAL_READER_SOURCE_PERMIT, semaphore); lastPrintTime.put(AGENT_GLOBAL_READER_SOURCE_PERMIT, 0L); + defaultSemaphoreTypes.add(AGENT_GLOBAL_READER_SOURCE_PERMIT); semaphore = new Semaphore( conf.getInt(AGENT_GLOBAL_READER_QUEUE_PERMIT, DEFAULT_AGENT_GLOBAL_READER_QUEUE_PERMIT)); semaphoreMap.put(AGENT_GLOBAL_READER_QUEUE_PERMIT, semaphore); lastPrintTime.put(AGENT_GLOBAL_READER_QUEUE_PERMIT, 0L); + defaultSemaphoreTypes.add(AGENT_GLOBAL_READER_QUEUE_PERMIT); semaphore = new Semaphore( conf.getInt(AGENT_GLOBAL_WRITER_PERMIT, DEFAULT_AGENT_GLOBAL_WRITER_PERMIT)); semaphoreMap.put(AGENT_GLOBAL_WRITER_PERMIT, semaphore); lastPrintTime.put(AGENT_GLOBAL_WRITER_PERMIT, 0L); + defaultSemaphoreTypes.add(AGENT_GLOBAL_WRITER_PERMIT); } /** @@ -78,6 +84,20 @@ public static MemoryManager getInstance() { return memoryManager; } + public void addSemaphore(String semaphoreType, int permit) { + if (semaphoreMap.containsKey(semaphoreType)) { + return; + } + synchronized (MemoryManager.class) { + if (semaphoreMap.containsKey(semaphoreType)) { + return; + } + Semaphore semaphore = new Semaphore(permit); + semaphoreMap.put(semaphoreType, semaphore); + lastPrintTime.put(semaphoreType, 0L); + } + } + public boolean tryAcquire(String semaphoreName, int permit) { Semaphore semaphore = semaphoreMap.get(semaphoreName); if (semaphore == null) { @@ -123,5 +143,10 @@ public void printAll() { printDetail(AGENT_GLOBAL_READER_SOURCE_PERMIT, "printAll"); printDetail(AGENT_GLOBAL_READER_QUEUE_PERMIT, "printAll"); printDetail(AGENT_GLOBAL_WRITER_PERMIT, "printAll"); + semaphoreMap.entrySet().forEach(entry -> { + if (!defaultSemaphoreTypes.contains(entry.getKey())) { + printDetail(entry.getKey(), "printAll"); + } + }); } } diff --git a/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/AgentBaseTestsHelper.java b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/AgentBaseTestsHelper.java index fa37fa2f1ce..af0a54f1295 100755 --- a/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/AgentBaseTestsHelper.java +++ b/inlong-agent/agent-core/src/test/java/org/apache/inlong/agent/core/AgentBaseTestsHelper.java @@ -18,12 +18,8 @@ package org.apache.inlong.agent.core; import org.apache.inlong.agent.conf.AgentConfiguration; -import org.apache.inlong.agent.conf.TaskProfile; import org.apache.inlong.agent.constant.AgentConstants; import org.apache.inlong.agent.constant.FetcherConstants; -import org.apache.inlong.agent.pojo.FileTask.FileTaskConfig; -import org.apache.inlong.common.enums.TaskStateEnum; -import org.apache.inlong.common.pojo.agent.DataConfig; import com.google.gson.Gson; import com.google.gson.GsonBuilder; @@ -74,33 +70,4 @@ public void teardownAgentHome() { } } } - - public TaskProfile getTaskProfile(int taskId, String pattern, boolean retry, Long startTime, Long endTime, - TaskStateEnum state, String timeZone) { - DataConfig dataConfig = getDataConfig(taskId, pattern, retry, startTime, endTime, state, timeZone); - TaskProfile profile = TaskProfile.convertToTaskProfile(dataConfig); - return profile; - } - - private DataConfig getDataConfig(int taskId, String pattern, boolean retry, Long startTime, Long endTime, - TaskStateEnum state, String timeZone) { - DataConfig dataConfig = new DataConfig(); - dataConfig.setInlongGroupId("testGroupId"); - dataConfig.setInlongStreamId("testStreamId"); - dataConfig.setDataReportType(1); - dataConfig.setTaskType(3); - dataConfig.setTaskId(taskId); - dataConfig.setTimeZone(timeZone); - dataConfig.setState(state.ordinal()); - FileTaskConfig fileTaskConfig = new FileTaskConfig(); - fileTaskConfig.setPattern(pattern); - fileTaskConfig.setTimeOffset("0h"); - fileTaskConfig.setMaxFileCount(100); - fileTaskConfig.setCycleUnit("h"); - fileTaskConfig.setRetry(retry); - fileTaskConfig.setStartTime(startTime); - fileTaskConfig.setEndTime(endTime); - dataConfig.setExtParams(GSON.toJson(fileTaskConfig)); - return dataConfig; - } } diff --git a/inlong-agent/agent-installer/bin/installer.sh b/inlong-agent/agent-installer/bin/installer.sh index 09a412348d7..729f0f2de7a 100755 --- a/inlong-agent/agent-installer/bin/installer.sh +++ b/inlong-agent/agent-installer/bin/installer.sh @@ -31,13 +31,27 @@ function help() { echo " help: get help from agent installer" } +function getPid() { + local process_name="installer.Main" + local user=$(whoami) + local pid=$(ps -u $user -f | grep 'java' | grep "$process_name" | grep -v grep | awk '{print $2}') + + if [ -z "$pid" ]; then + echo "No matching process found." + return 1 + fi + + echo "$pid" + return 0 +} + function running() { - process=$("$JPS" -l| grep "installer.Main" | grep -v grep) - if [ "${process}" = "" ]; then - return 1; - else - return 0; - fi + pid=$(getPid) + if [ $? -eq 0 ]; then + return 0 + else + return 1 + fi } # start installer @@ -56,7 +70,7 @@ function stop_installer() { exit 1 fi count=0 - pid=$("$JPS" -l| grep "installer.Main"| grep -v grep | awk '{print $1}') + pid=$(getPid) while running; do (( count++ )) diff --git a/inlong-agent/agent-installer/src/main/java/org/apache/inlong/agent/installer/ModuleManager.java b/inlong-agent/agent-installer/src/main/java/org/apache/inlong/agent/installer/ModuleManager.java index de05a9d0f72..cae8b25f688 100755 --- a/inlong-agent/agent-installer/src/main/java/org/apache/inlong/agent/installer/ModuleManager.java +++ b/inlong-agent/agent-installer/src/main/java/org/apache/inlong/agent/installer/ModuleManager.java @@ -551,8 +551,17 @@ private boolean downloadModule(ModuleConfig module) { authHeader.forEach((k, v) -> { conn.setRequestProperty(k, v); }); - String path = - module.getPackageConfig().getStoragePath() + "/" + module.getPackageConfig().getFileName(); + String saveFolder = getRealPath(module.getPackageConfig().getStoragePath()); + File folder = new File(saveFolder); + if (!folder.exists()) { + boolean folderCreated = folder.mkdirs(); + if (folderCreated) { + LOGGER.info("saveFolder {} created", saveFolder); + } else { + LOGGER.error("failed to create saveFolder {}", saveFolder); + } + } + String path = saveFolder + "/" + module.getPackageConfig().getFileName(); try (InputStream inputStream = conn.getInputStream(); FileOutputStream outputStream = new FileOutputStream(path)) { LOGGER.info("module {}({}) save path {}", module.getId(), module.getName(), path); @@ -578,7 +587,8 @@ private boolean downloadModule(ModuleConfig module) { } private boolean isPackageDownloaded(ModuleConfig module) { - String path = module.getPackageConfig().getStoragePath() + "/" + module.getPackageConfig().getFileName(); + String path = + getRealPath(module.getPackageConfig().getStoragePath()) + "/" + module.getPackageConfig().getFileName(); String fileMd5 = calcFileMd5(path); if (Objects.equals(fileMd5, module.getPackageConfig().getMd5())) { return true; @@ -589,6 +599,15 @@ private boolean isPackageDownloaded(ModuleConfig module) { } } + private String getRealPath(String originPath) { + String homeDir = System.getProperty("user.home"); + if (homeDir == null) { + LOGGER.warn("user.home should not be null"); + return originPath; + } + return originPath.replace("~", homeDir).replace("${HOME}", homeDir).replace("${home}", homeDir); + } + @Override public void start() throws Exception { httpManager = getHttpManager(conf); diff --git a/inlong-agent/agent-plugins/pom.xml b/inlong-agent/agent-plugins/pom.xml index 4c77f84dc31..863e285ab2f 100644 --- a/inlong-agent/agent-plugins/pom.xml +++ b/inlong-agent/agent-plugins/pom.xml @@ -92,6 +92,11 @@ kafka-clients ${kafka.clients.version} + + com.qcloud + cos_api + ${cos.sdk.version} + io.debezium debezium-connector-mysql diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerFetcher.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerFetcher.java index e0125751c37..3ab63b0118a 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerFetcher.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/fetcher/ManagerFetcher.java @@ -28,6 +28,7 @@ import org.apache.inlong.agent.utils.HttpManager; import org.apache.inlong.agent.utils.ThreadUtils; import org.apache.inlong.common.enums.PullJobTypeEnum; +import org.apache.inlong.common.enums.TaskTypeEnum; import org.apache.inlong.common.pojo.agent.AgentConfigInfo; import org.apache.inlong.common.pojo.agent.AgentConfigRequest; import org.apache.inlong.common.pojo.agent.AgentResponseCode; @@ -42,10 +43,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.text.ParseException; -import java.text.SimpleDateFormat; import java.util.ArrayList; -import java.util.Date; import java.util.List; import static org.apache.inlong.agent.constant.AgentConstants.AGENT_CLUSTER_NAME; @@ -225,32 +223,22 @@ private Runnable configFetchThread() { private TaskResult getTestConfig(String testDir, int normalTaskId, int retryTaskId, int state) { List configs = new ArrayList<>(); - String startStr = "2023-07-10 00:00:00"; - String endStr = "2023-07-22 00:00:00"; - Long start = 0L; - Long end = 0L; String normalPattern = testDir + "YYYY/YYYYMMDDhhmm_2.log_[0-9]+"; String retryPattern = testDir + "YYYY/YYYYMMDD_1.log_[0-9]+"; - try { - Date parse = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").parse(startStr); - start = parse.getTime(); - parse = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").parse(endStr); - end = parse.getTime(); - } catch (ParseException e) { - e.printStackTrace(); - } - configs.add(getTestDataConfig(normalTaskId, normalPattern, false, start, end, CycleUnitType.MINUTE, state)); - configs.add(getTestDataConfig(retryTaskId, retryPattern, true, start, end, CycleUnitType.DAY, state)); + configs.add(getTestDataConfig(normalTaskId, normalPattern, false, "202307100000", "202307220000", + CycleUnitType.MINUTE, state)); + configs.add( + getTestDataConfig(retryTaskId, retryPattern, true, "20230710", "20230722", CycleUnitType.DAY, state)); return TaskResult.builder().dataConfigs(configs).build(); } - private DataConfig getTestDataConfig(int taskId, String pattern, boolean retry, Long startTime, Long endTime, + private DataConfig getTestDataConfig(int taskId, String pattern, boolean retry, String startTime, String endTime, String cycleUnit, int state) { DataConfig dataConfig = new DataConfig(); dataConfig.setInlongGroupId("devcloud_group_id"); dataConfig.setInlongStreamId("devcloud_stream_id"); dataConfig.setDataReportType(0); - dataConfig.setTaskType(3); + dataConfig.setTaskType(TaskTypeEnum.FILE.getType()); dataConfig.setTaskId(taskId); dataConfig.setState(state); dataConfig.setTimeZone("GMT+8:00"); @@ -260,8 +248,8 @@ private DataConfig getTestDataConfig(int taskId, String pattern, boolean retry, fileTaskConfig.setMaxFileCount(100); fileTaskConfig.setCycleUnit(cycleUnit); fileTaskConfig.setRetry(retry); - fileTaskConfig.setStartTime(startTime); - fileTaskConfig.setEndTime(endTime); + fileTaskConfig.setDataTimeFrom(startTime); + fileTaskConfig.setDataTimeTo(endTime); fileTaskConfig.setDataContentStyle("CSV"); fileTaskConfig.setDataSeparator("|"); dataConfig.setExtParams(GSON.toJson(fileTaskConfig)); diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/LoadBalance.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/COSInstance.java similarity index 61% rename from inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/LoadBalance.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/COSInstance.java index e71838ae76d..6f40f727dd0 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/LoadBalance.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/COSInstance.java @@ -15,30 +15,21 @@ * limitations under the License. */ -package org.apache.inlong.sdk.dataproxy; +package org.apache.inlong.agent.plugin.instance; -public enum LoadBalance { +import org.apache.inlong.agent.conf.InstanceProfile; +import org.apache.inlong.agent.constant.TaskConstants; - RANDOM("random", 0), - ROBIN("robin", 1), - CONSISTENCY_HASH("consistency hash", 2), - WEIGHT_RANDOM("weight random", 3), - WEIGHT_ROBIN("weight robin", 4); +import java.io.IOException; - private final String name; - private final int index; - - LoadBalance(String name, int index) { - this.name = name; - this.index = index; - } - - public String getName() { - return name; - } +/** + * cos instance contains source and sink. + * main job is to read from source and write to sink + */ +public class COSInstance extends CommonInstance { - public int getIndex() { - return index; + @Override + public void setInodeInfo(InstanceProfile profile) throws IOException { + profile.set(TaskConstants.INODE_INFO, ""); } - } diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/CommonInstance.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/CommonInstance.java index 415b05825af..6cc97a159c7 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/CommonInstance.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/CommonInstance.java @@ -95,20 +95,34 @@ public boolean init(Object srcManager, InstanceProfile srcProfile) { @Override public void destroy() { - if (!inited) { - return; - } - doChangeState(State.SUCCEEDED); + Long start = AgentUtils.getCurrentTime(); + notifyDestroy(); while (running) { AgentUtils.silenceSleepInMs(DESTROY_LOOP_WAIT_TIME_MS); } + LOGGER.info("destroy instance wait run elapse {} ms instance {}", AgentUtils.getCurrentTime() - start, + profile.getInstanceId()); + start = AgentUtils.getCurrentTime(); this.source.destroy(); + LOGGER.info("destroy instance wait source elapse {} ms instance {}", AgentUtils.getCurrentTime() - start, + profile.getInstanceId()); + start = AgentUtils.getCurrentTime(); this.sink.destroy(); + LOGGER.info("destroy instance wait sink elapse {} ms instance {}", AgentUtils.getCurrentTime() - start, + profile.getInstanceId()); + } + + @Override + public void notifyDestroy() { + if (!inited) { + return; + } + doChangeState(State.SUCCEEDED); } @Override public void run() { - Thread.currentThread().setName("file-instance-core-" + getTaskId() + "-" + getInstanceId()); + Thread.currentThread().setName("instance-core-" + getTaskId() + "-" + getInstanceId()); running = true; try { doRun(); diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/FileInstance.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/FileInstance.java index 3c86a4c33ff..3a4225abdf3 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/FileInstance.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/instance/FileInstance.java @@ -19,7 +19,7 @@ import org.apache.inlong.agent.conf.InstanceProfile; import org.apache.inlong.agent.constant.TaskConstants; -import org.apache.inlong.agent.plugin.utils.file.FileDataUtils; +import org.apache.inlong.agent.plugin.task.logcollection.local.FileDataUtils; import java.io.IOException; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sinks/ProxySink.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sinks/ProxySink.java index b7ddc79a15d..e00ad65cbab 100755 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sinks/ProxySink.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sinks/ProxySink.java @@ -57,8 +57,9 @@ public class ProxySink extends AbstractSink { private static final Logger LOGGER = LoggerFactory.getLogger(ProxySink.class); - private final int DESTROY_LOOP_WAIT_TIME_MS = 10; + private final int LOOP_WAIT_TIME_MS = 10; public final int SAVE_OFFSET_INTERVAL_MS = 1000; + public volatile long lastFlushOffset = AgentUtils.getCurrentTime(); private static final ThreadPoolExecutor EXECUTOR_SERVICE = new ThreadPoolExecutor( 0, Integer.MAX_VALUE, 1L, TimeUnit.SECONDS, @@ -193,12 +194,21 @@ public void destroy() { if (!inited) { return; } + Long start = AgentUtils.getCurrentTime(); shutdown = true; + senderManager.Stop(); + LOGGER.info("destroy proxySink, wait for sender close {} ms instance {}", AgentUtils.getCurrentTime() - start, + profile.getInstanceId()); + start = AgentUtils.getCurrentTime(); while (running || offsetRunning) { - AgentUtils.silenceSleepInMs(DESTROY_LOOP_WAIT_TIME_MS); + AgentUtils.silenceSleepInMs(LOOP_WAIT_TIME_MS); } - senderManager.Stop(); + LOGGER.info("destroy proxySink, wait for run close {} ms instance {}", AgentUtils.getCurrentTime() - start, + profile.getInstanceId()); + start = AgentUtils.getCurrentTime(); clearOffset(); + LOGGER.info("destroy proxySink, wait for offset clear {} ms instance {}", AgentUtils.getCurrentTime() - start, + profile.getInstanceId()); LOGGER.info("destroy sink {} end", sourceName); } @@ -234,8 +244,11 @@ private Runnable flushOffset() { LOGGER.info("start flush offset {}:{}", inlongGroupId, sourceName); offsetRunning = true; while (!shutdown) { - doFlushOffset(); - AgentUtils.silenceSleepInMs(SAVE_OFFSET_INTERVAL_MS); + if (AgentUtils.getCurrentTime() - lastFlushOffset > SAVE_OFFSET_INTERVAL_MS) { + doFlushOffset(); + lastFlushOffset = AgentUtils.getCurrentTime(); + } + AgentUtils.silenceSleepInMs(LOOP_WAIT_TIME_MS); } LOGGER.info("stop flush offset {}:{}", inlongGroupId, sourceName); offsetRunning = false; @@ -269,6 +282,7 @@ private void doFlushOffset() { } private void clearOffset() { + doFlushOffset(); packageAckInfoLock.writeLock().lock(); for (int i = 0; i < ackInfoList.size();) { MemoryManager.getInstance().release(AGENT_GLOBAL_WRITER_PERMIT, ackInfoList.remove(i).getLen()); diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sinks/filecollect/SenderManager.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sinks/filecollect/SenderManager.java index a37a171a372..ec4502a7fb7 100755 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sinks/filecollect/SenderManager.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sinks/filecollect/SenderManager.java @@ -70,6 +70,7 @@ public class SenderManager { private static final Logger LOGGER = LoggerFactory.getLogger(SenderManager.class); private static final SequentialID SEQUENTIAL_ID = SequentialID.getInstance(); + public static final int RESEND_QUEUE_WAIT_MS = 10; // cache for group and sender list, share the map cross agent lifecycle. private DefaultMessageSender sender; private LinkedBlockingQueue resendQueue; @@ -172,9 +173,12 @@ public void Stop() { } private void closeMessageSender() { + Long start = AgentUtils.getCurrentTime(); if (sender != null) { sender.close(); } + LOGGER.info("close sender elapse {} ms instance {}", AgentUtils.getCurrentTime() - start, + profile.getInstanceId()); } private AgentMetricItem getMetricItem(Map otherDimensions) { @@ -286,7 +290,7 @@ private Runnable flushResendQueue() { resendRunning = true; while (!shutdown) { try { - AgentSenderCallback callback = resendQueue.poll(1, TimeUnit.SECONDS); + AgentSenderCallback callback = resendQueue.poll(RESEND_QUEUE_WAIT_MS, TimeUnit.MILLISECONDS); if (callback != null) { SenderMessage message = callback.message; AuditUtils.add(AuditUtils.AUDIT_ID_AGENT_RESEND, message.getGroupId(), diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/COSSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/COSSource.java new file mode 100755 index 00000000000..b6792ac82d8 --- /dev/null +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/COSSource.java @@ -0,0 +1,356 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.agent.plugin.sources; + +import org.apache.inlong.agent.common.AgentThreadFactory; +import org.apache.inlong.agent.conf.AgentConfiguration; +import org.apache.inlong.agent.conf.InstanceProfile; +import org.apache.inlong.agent.conf.OffsetProfile; +import org.apache.inlong.agent.constant.TaskConstants; +import org.apache.inlong.agent.core.FileStaticManager; +import org.apache.inlong.agent.core.FileStaticManager.FileStatic; +import org.apache.inlong.agent.core.task.MemoryManager; +import org.apache.inlong.agent.core.task.OffsetManager; +import org.apache.inlong.agent.except.FileException; +import org.apache.inlong.agent.metrics.audit.AuditUtils; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; +import org.apache.inlong.agent.plugin.sources.file.AbstractSource; +import org.apache.inlong.agent.plugin.utils.cos.COSUtils; +import org.apache.inlong.agent.utils.AgentUtils; +import org.apache.inlong.agent.utils.ThreadUtils; + +import com.qcloud.cos.COSClient; +import com.qcloud.cos.model.COSObject; +import com.qcloud.cos.model.GetObjectRequest; +import com.qcloud.cos.model.ObjectMetadata; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static org.apache.inlong.agent.constant.TaskConstants.COS_CONTENT_STYLE; + +/** + * Read COS files + */ +public class COSSource extends AbstractSource { + + public static final int LEN_OF_FILE_OFFSET_ARRAY = 2; + public static final String AGENT_GLOBAL_COS_SOURCE_PERMIT = "agent.global.cos.source.permit"; + public static final int DEFAULT_AGENT_GLOBAL_COS_SOURCE_PERMIT = 128 * 1000 * 1000; + + @Data + @AllArgsConstructor + @NoArgsConstructor + protected class FileOffset { + + private Long lineOffset; + private Long byteOffset; + private boolean hasByteOffset; + } + + private static final Logger LOGGER = LoggerFactory.getLogger(COSSource.class); + public static final String OFFSET_SEP = ":"; + protected final Integer WAIT_TIMEOUT_MS = 10; + private final Integer SIZE_OF_BUFFER_TO_READ_FILE = 1024 * 1024; + private final Long META_UPDATE_INTERVAL_MS = 10000L; + private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + + private String fileName; + private byte[] bufferToReadFile; + public volatile long linePosition = 0; + public volatile long bytePosition = 0; + private volatile boolean fileExist = true; + private volatile long lastInodeUpdateTime = 0; + private COSClient cosClient; + private String bucketName; + private String secretId; + private String secretKey; + private String strRegion; + private ObjectMetadata metadata; + protected BlockingQueue queue; + private static final ThreadPoolExecutor EXECUTOR_SERVICE = new ThreadPoolExecutor( + 0, Integer.MAX_VALUE, + 1L, TimeUnit.SECONDS, + new SynchronousQueue<>(), + new AgentThreadFactory("cos-source-pool")); + private volatile boolean running = false; + + public COSSource() { + } + + @Override + protected void initExtendClass() { + extendClass = DefaultExtendedHandler.class.getCanonicalName(); + } + + @Override + protected void initSource(InstanceProfile profile) { + try { + String offset = ""; + if (offsetProfile != null) { + offset = offsetProfile.toJsonStr(); + } + LOGGER.info("LogFileSource init: {} offset: {}", profile.toJsonStr(), offset); + AgentConfiguration conf = AgentConfiguration.getAgentConf(); + int permit = conf.getInt(AGENT_GLOBAL_COS_SOURCE_PERMIT, DEFAULT_AGENT_GLOBAL_COS_SOURCE_PERMIT); + MemoryManager.getInstance().addSemaphore(AGENT_GLOBAL_COS_SOURCE_PERMIT, permit); + fileName = profile.getInstanceId(); + bucketName = profile.get(TaskConstants.COS_TASK_BUCKET_NAME); + secretId = profile.get(TaskConstants.COS_TASK_SECRET_ID); + secretKey = profile.get(TaskConstants.COS_TASK_SECRET_KEY); + strRegion = profile.get(TaskConstants.COS_TASK_REGION); + cosClient = COSUtils.createCli(secretId, secretKey, strRegion); + metadata = cosClient.getObjectMetadata(bucketName, fileName); + queue = new LinkedBlockingQueue<>(CACHE_QUEUE_SIZE); + bufferToReadFile = new byte[SIZE_OF_BUFFER_TO_READ_FILE]; + lastInodeUpdateTime = AgentUtils.getCurrentTime(); + initOffset(taskId); + EXECUTOR_SERVICE.execute(run()); + } catch (Exception ex) { + stopRunning(); + throw new FileException("error init stream for " + fileName, ex); + } + } + + @Override + protected boolean doPrepareToRead() { + if (AgentUtils.getCurrentTime() - lastInodeUpdateTime > META_UPDATE_INTERVAL_MS) { + metadata = cosClient.getObjectMetadata(bucketName, fileName); + lastInodeUpdateTime = AgentUtils.getCurrentTime(); + } + if (metadata.getContentLength() < bytePosition) { + fileExist = false; + LOGGER.info("file rotate, instance will restart and offset will be clean, file {}", + fileName); + return false; + } + return true; + } + + @Override + protected List readFromSource() { + if (queue.isEmpty()) { + return null; + } + int count = 0; + int len = 0; + List lines = new ArrayList<>(); + while (!queue.isEmpty() && count < BATCH_READ_LINE_COUNT && len < BATCH_READ_LINE_TOTAL_LEN) { + if (len + queue.peek().getData().length > BATCH_READ_LINE_TOTAL_LEN) { + break; + } + len += queue.peek().getData().length; + count++; + lines.add(queue.poll()); + } + MemoryManager.getInstance().release(AGENT_GLOBAL_COS_SOURCE_PERMIT, len); + return lines; + } + + @Override + protected void printCurrentState() { + LOGGER.info("path is {}, linePosition {}, bytePosition is {} file len {}", fileName, linePosition, + bytePosition, metadata.getContentLength()); + } + + @Override + protected String getThreadName() { + return "cos-file-source-" + taskId + "-" + fileName; + } + + private void initOffset(String taskId) { + long lineOffset; + long byteOffset; + if (offsetProfile != null) { + FileOffset fileOffset = parseFIleOffset(offsetProfile.getOffset()); + lineOffset = fileOffset.lineOffset; + byteOffset = fileOffset.byteOffset; + LOGGER.info("initOffset inode no change taskId {} restore lineOffset {} byteOffset {}, file {}", taskId, + lineOffset, byteOffset, fileName); + } else { + lineOffset = 0; + byteOffset = 0; + LOGGER.info("initOffset taskId {} for new all read lineOffset {} byteOffset {} file {}", taskId, + lineOffset, byteOffset, fileName); + } + linePosition = lineOffset; + bytePosition = byteOffset; + } + + private Runnable run() { + return () -> { + AgentThreadFactory.nameThread(getThreadName()); + running = true; + try { + doRun(); + } catch (Throwable e) { + LOGGER.error("do run error maybe file deleted: ", e); + ThreadUtils.threadThrowableHandler(Thread.currentThread(), e); + } + running = false; + }; + } + + /** + * Read new lines. + * + * @return The new position after the lines have been read + * @throws IOException if an I/O error occurs. + */ + protected void doRun() throws IOException { + GetObjectRequest getObjectRequest = new GetObjectRequest(bucketName, fileName); + getObjectRequest.setRange(bytePosition, metadata.getContentLength()); + COSObject cosObject = cosClient.getObject(getObjectRequest); + InputStream inputStream = cosObject.getObjectContent(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + int num; + boolean overLen = false; + while ((num = inputStream.read(bufferToReadFile)) != -1) { + LOGGER.debug("read size {}", num); + for (int i = 0; i < num; i++) { + byte ch = bufferToReadFile[i]; + bytePosition++; + switch (ch) { + case '\n': + linePosition++; + boolean suc = false; + while (isRunnable() && !suc) { + SourceData sourceData = new SourceData(baos.toByteArray(), + getOffsetString(linePosition, bytePosition)); + boolean suc4Queue = waitForPermit(AGENT_GLOBAL_COS_SOURCE_PERMIT, + sourceData.getData().length); + if (!suc4Queue) { + break; + } + suc = queue.offer(sourceData); + if (!suc) { + MemoryManager.getInstance() + .release(AGENT_GLOBAL_COS_SOURCE_PERMIT, sourceData.getData().length); + AgentUtils.silenceSleepInMs(WAIT_TIMEOUT_MS); + } + } + if (overLen) { + LOGGER.warn("readLines over len finally string len {}", + new String(baos.toByteArray()).length()); + long auditTime = 0; + auditTime = profile.getSinkDataTime(); + AuditUtils.add(AuditUtils.AUDIT_ID_AGENT_READ_FAILED, inlongGroupId, inlongStreamId, + auditTime, 1, maxPackSize, auditVersion); + AuditUtils.add(AuditUtils.AUDIT_ID_AGENT_READ_FAILED_REAL_TIME, inlongGroupId, + inlongStreamId, AgentUtils.getCurrentTime(), 1, maxPackSize, auditVersion); + } + baos.reset(); + overLen = false; + break; + case '\r': + break; + default: + if (baos.size() < maxPackSize) { + baos.write(ch); + } else { + overLen = true; + } + } + } + } + baos.close(); + inputStream.close(); + cosObject.close(); + } + + private String getOffsetString(Long lineOffset, Long byteOffset) { + return lineOffset + OFFSET_SEP + byteOffset; + } + + private FileOffset parseFIleOffset(String offset) { + String[] offsetArray = offset.split(OFFSET_SEP); + if (offsetArray.length == LEN_OF_FILE_OFFSET_ARRAY) { + return new FileOffset(Long.parseLong(offsetArray[0]), Long.parseLong(offsetArray[1]), true); + } else { + return new FileOffset(Long.parseLong(offsetArray[0]), null, false); + } + } + + @Override + protected boolean isRunnable() { + return runnable && fileExist; + } + + @Override + public boolean sourceExist() { + return fileExist; + } + + @Override + protected void releaseSource() { + while (running) { + AgentUtils.silenceSleepInMs(1); + } + if (cosClient != null) { + FileStatic data = new FileStatic(); + data.setTaskId(taskId); + data.setRetry(String.valueOf(profile.isRetry())); + data.setContentType(profile.get(COS_CONTENT_STYLE)); + data.setGroupId(profile.getInlongGroupId()); + data.setStreamId(profile.getInlongStreamId()); + data.setDataTime(format.format(profile.getSinkDataTime())); + data.setFileName(profile.getInstanceId()); + data.setFileLen(String.valueOf(metadata.getContentLength())); + data.setReadBytes(String.valueOf(bytePosition)); + data.setReadLines(String.valueOf(linePosition)); + OffsetProfile offsetProfile = OffsetManager.getInstance().getOffset(taskId, instanceId); + if (offsetProfile != null) { + data.setSendLines(offsetProfile.getOffset()); + } + FileStaticManager.putStaticMsg(data); + cosClient.shutdown(); + } + while (!queue.isEmpty()) { + MemoryManager.getInstance().release(AGENT_GLOBAL_COS_SOURCE_PERMIT, queue.poll().getData().length); + } + } + + private boolean waitForPermit(String permitName, int permitLen) { + boolean suc = false; + while (!suc) { + suc = MemoryManager.getInstance().tryAcquire(permitName, permitLen); + if (!suc) { + MemoryManager.getInstance().printDetail(permitName, "cos_source"); + if (!isRunnable()) { + return false; + } + AgentUtils.silenceSleepInMs(WAIT_TIMEOUT_MS); + } + } + return true; + } +} diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/DatabaseSqlSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/DatabaseSqlSource.java index d2d58a6f291..44541a200f5 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/DatabaseSqlSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/DatabaseSqlSource.java @@ -19,8 +19,8 @@ import org.apache.inlong.agent.conf.InstanceProfile; import org.apache.inlong.agent.plugin.Message; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/KafkaSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/KafkaSource.java index b9a4494bb75..e6ac7a1fae8 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/KafkaSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/KafkaSource.java @@ -19,8 +19,8 @@ import org.apache.inlong.agent.conf.InstanceProfile; import org.apache.inlong.agent.except.FileException; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; import org.apache.commons.lang3.ObjectUtils; import org.apache.commons.lang3.StringUtils; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/LogFileSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/LogFileSource.java index 9ce20f6daa3..6774751a342 100755 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/LogFileSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/LogFileSource.java @@ -26,11 +26,14 @@ import org.apache.inlong.agent.core.task.OffsetManager; import org.apache.inlong.agent.except.FileException; import org.apache.inlong.agent.metrics.audit.AuditUtils; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; -import org.apache.inlong.agent.plugin.utils.file.FileDataUtils; +import org.apache.inlong.agent.plugin.task.logcollection.local.FileDataUtils; import org.apache.inlong.agent.utils.AgentUtils; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,17 +48,30 @@ import java.util.ArrayList; import java.util.List; -import static org.apache.inlong.agent.constant.TaskConstants.SOURCE_DATA_CONTENT_STYLE; +import static org.apache.inlong.agent.constant.TaskConstants.FILE_CONTENT_STYLE; /** * Read text files */ public class LogFileSource extends AbstractSource { + public static final int LEN_OF_FILE_OFFSET_ARRAY = 2; + + @Data + @AllArgsConstructor + @NoArgsConstructor + protected class FileOffset { + + private Long lineOffset; + private Long byteOffset; + private boolean hasByteOffset; + } + private static final Logger LOGGER = LoggerFactory.getLogger(LogFileSource.class); + public static final String OFFSET_SEP = ":"; private final Integer SIZE_OF_BUFFER_TO_READ_FILE = 64 * 1024; private final Long INODE_UPDATE_INTERVAL_MS = 1000L; - private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); // 设置格式 + private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); private String fileName; private File file; @@ -86,8 +102,7 @@ protected void initSource(InstanceProfile profile) { file = new File(fileName); inodeInfo = profile.get(TaskConstants.INODE_INFO); lastInodeUpdateTime = AgentUtils.getCurrentTime(); - linePosition = getInitLineOffset(isIncrement, taskId, instanceId, inodeInfo); - bytePosition = getBytePositionByLine(linePosition); + initOffset(isIncrement, taskId, instanceId, inodeInfo); randomAccessFile = new RandomAccessFile(file, "r"); } catch (Exception ex) { stopRunning(); @@ -137,14 +152,9 @@ protected String getThreadName() { } private List readFromPos(long pos) throws IOException { - List lines = new ArrayList<>(); - List dataList = new ArrayList<>(); - bytePosition = readLines(randomAccessFile, pos, lines, BATCH_READ_LINE_COUNT, BATCH_READ_LINE_TOTAL_LEN, false); - for (int i = 0; i < lines.size(); i++) { - linePosition++; - dataList.add(new SourceData(lines.get(i), Long.toString(linePosition))); - } - return dataList; + List lines = new ArrayList<>(); + bytePosition = readLines(randomAccessFile, pos, lines, BATCH_READ_LINE_COUNT, BATCH_READ_LINE_TOTAL_LEN); + return lines; } private int getRealLineCount(String fileName) { @@ -157,30 +167,39 @@ private int getRealLineCount(String fileName) { } } - private long getInitLineOffset(boolean isIncrement, String taskId, String instanceId, String inodeInfo) { - long offset = 0; + private void initOffset(boolean isIncrement, String taskId, String instanceId, String inodeInfo) + throws IOException { + long lineOffset; + long byteOffset; if (offsetProfile != null && offsetProfile.getInodeInfo().compareTo(inodeInfo) == 0) { - offset = Long.parseLong(offsetProfile.getOffset()); - int fileLineCount = getRealLineCount(instanceId); - if (fileLineCount < offset) { - LOGGER.info("getInitLineOffset inode no change taskId {} file rotate, offset set to 0, file {}", taskId, - fileName); - offset = 0; + FileOffset fileOffset = parseFIleOffset(offsetProfile.getOffset()); + if (fileOffset.hasByteOffset) { + lineOffset = fileOffset.lineOffset; + byteOffset = fileOffset.byteOffset; + LOGGER.info("initOffset inode no change taskId {} restore lineOffset {} byteOffset {}, file {}", taskId, + lineOffset, byteOffset, fileName); } else { - LOGGER.info("getInitLineOffset inode no change taskId {} from offset store {}, file {}", taskId, offset, - fileName); + lineOffset = fileOffset.lineOffset; + byteOffset = getBytePositionByLine(lineOffset); + LOGGER.info("initOffset inode no change taskId {} restore lineOffset {} count byteOffset {}, file {}", + taskId, + lineOffset, byteOffset, fileName); } } else { if (isIncrement) { - offset = getRealLineCount(instanceId); - LOGGER.info("getInitLineOffset taskId {} for new increment read from {} file {}", taskId, - offset, fileName); + lineOffset = getRealLineCount(instanceId); + byteOffset = getBytePositionByLine(lineOffset); + LOGGER.info("initOffset taskId {} for new increment lineOffset {} byteOffset {}, file {}", taskId, + lineOffset, byteOffset, fileName); } else { - offset = 0; - LOGGER.info("getInitLineOffset taskId {} for new all read from 0 file {}", taskId, fileName); + lineOffset = 0; + byteOffset = 0; + LOGGER.info("initOffset taskId {} for new all read lineOffset {} byteOffset {} file {}", taskId, + lineOffset, byteOffset, fileName); } } - return offset; + linePosition = lineOffset; + bytePosition = byteOffset; } public File getFile() { @@ -202,9 +221,9 @@ private long getBytePositionByLine(long linePosition) throws IOException { try { input = new RandomAccessFile(file, "r"); while (readCount < linePosition) { - List lines = new ArrayList<>(); + List lines = new ArrayList<>(); pos = readLines(input, pos, lines, Math.min((int) (linePosition - readCount), BATCH_READ_LINE_COUNT), - BATCH_READ_LINE_TOTAL_LEN, true); + BATCH_READ_LINE_TOTAL_LEN); readCount += lines.size(); if (lines.size() == 0) { LOGGER.error("getBytePositionByLine LineNum {} larger than the real file"); @@ -229,8 +248,8 @@ private long getBytePositionByLine(long linePosition) throws IOException { * @return The new position after the lines have been read * @throws IOException if an I/O error occurs. */ - private long readLines(RandomAccessFile reader, long pos, List lines, int maxLineCount, int maxLineTotalLen, - boolean isCounting) + private long readLines(RandomAccessFile reader, long pos, List lines, int maxLineCount, + int maxLineTotalLen) throws IOException { if (maxLineCount == 0) { return pos; @@ -248,13 +267,10 @@ private long readLines(RandomAccessFile reader, long pos, List lines, in byte ch = bufferToReadFile[i]; switch (ch) { case '\n': - if (isCounting) { - lines.add(null); - } else { - lines.add(baos.toByteArray()); - lineTotalLen += baos.size(); - } + linePosition++; rePos = pos + i + 1; + lines.add(new SourceData(baos.toByteArray(), getOffsetString(linePosition, rePos))); + lineTotalLen += baos.size(); if (overLen) { LOGGER.warn("readLines over len finally string len {}", new String(baos.toByteArray()).length()); @@ -297,6 +313,19 @@ private long readLines(RandomAccessFile reader, long pos, List lines, in return rePos; } + private String getOffsetString(Long lineOffset, Long byteOffset) { + return lineOffset + OFFSET_SEP + byteOffset; + } + + private FileOffset parseFIleOffset(String offset) { + String[] offsetArray = offset.split(OFFSET_SEP); + if (offsetArray.length == LEN_OF_FILE_OFFSET_ARRAY) { + return new FileOffset(Long.parseLong(offsetArray[0]), Long.parseLong(offsetArray[1]), true); + } else { + return new FileOffset(Long.parseLong(offsetArray[0]), null, false); + } + } + private boolean isInodeChanged() { if (AgentUtils.getCurrentTime() - lastInodeUpdateTime > INODE_UPDATE_INTERVAL_MS) { try { @@ -326,7 +355,7 @@ protected void releaseSource() { FileStatic data = new FileStatic(); data.setTaskId(taskId); data.setRetry(String.valueOf(profile.isRetry())); - data.setContentType(profile.get(SOURCE_DATA_CONTENT_STYLE)); + data.setContentType(profile.get(FILE_CONTENT_STYLE)); data.setGroupId(profile.getInlongGroupId()); data.setStreamId(profile.getInlongStreamId()); data.setDataTime(format.format(profile.getSinkDataTime())); @@ -335,10 +364,9 @@ protected void releaseSource() { data.setReadBytes(String.valueOf(bytePosition)); data.setReadLines(String.valueOf(linePosition)); OffsetProfile offsetProfile = OffsetManager.getInstance().getOffset(taskId, instanceId); - if (offsetProfile == null) { - return; + if (offsetProfile != null) { + data.setSendLines(offsetProfile.getOffset()); } - data.setSendLines(offsetProfile.getOffset()); FileStaticManager.putStaticMsg(data); randomAccessFile.close(); } catch (IOException e) { diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/MongoDBSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/MongoDBSource.java index c86b6392325..0afcfcc4d74 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/MongoDBSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/MongoDBSource.java @@ -23,8 +23,8 @@ import org.apache.inlong.agent.constant.AgentConstants; import org.apache.inlong.agent.constant.TaskConstants; import org.apache.inlong.agent.except.FileException; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; import io.debezium.connector.mongodb.MongoDbConnector; import io.debezium.connector.mongodb.MongoDbConnectorConfig; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/MqttSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/MqttSource.java index 960c405e68d..e3035d469e9 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/MqttSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/MqttSource.java @@ -22,8 +22,8 @@ import org.apache.inlong.agent.except.FileException; import org.apache.inlong.agent.message.DefaultMessage; import org.apache.inlong.agent.plugin.Message; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken; import org.eclipse.paho.client.mqttv3.MqttCallback; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/OracleSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/OracleSource.java index 748024dfe11..4e9b71e2536 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/OracleSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/OracleSource.java @@ -23,8 +23,8 @@ import org.apache.inlong.agent.constant.AgentConstants; import org.apache.inlong.agent.except.FileException; import org.apache.inlong.agent.plugin.Message; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; import io.debezium.connector.oracle.OracleConnector; import io.debezium.connector.oracle.OracleConnectorConfig; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PostgreSQLSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PostgreSQLSource.java index 642ae11edac..a75e6702e09 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PostgreSQLSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PostgreSQLSource.java @@ -23,8 +23,8 @@ import org.apache.inlong.agent.constant.AgentConstants; import org.apache.inlong.agent.constant.TaskConstants; import org.apache.inlong.agent.except.FileException; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; import io.debezium.connector.postgresql.PostgresConnector; import io.debezium.connector.postgresql.PostgresConnectorConfig; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PulsarSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PulsarSource.java index 501b9fb9c83..59d438105ba 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PulsarSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PulsarSource.java @@ -19,8 +19,8 @@ import org.apache.inlong.agent.conf.InstanceProfile; import org.apache.inlong.agent.except.FileException; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; import org.apache.commons.lang3.ObjectUtils; import org.apache.pulsar.client.api.Consumer; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/RedisSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/RedisSource.java index 9ef57d167a5..47f09f97825 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/RedisSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/RedisSource.java @@ -21,8 +21,8 @@ import org.apache.inlong.agent.conf.InstanceProfile; import org.apache.inlong.agent.constant.TaskConstants; import org.apache.inlong.agent.metrics.audit.AuditUtils; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; import com.google.common.collect.Maps; import com.google.gson.Gson; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/SQLServerSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/SQLServerSource.java index 1770b231e15..02d3d9bbda6 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/SQLServerSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/SQLServerSource.java @@ -24,8 +24,8 @@ import org.apache.inlong.agent.constant.SqlServerConstants; import org.apache.inlong.agent.constant.TaskConstants; import org.apache.inlong.agent.except.FileException; +import org.apache.inlong.agent.plugin.sources.extend.DefaultExtendedHandler; import org.apache.inlong.agent.plugin.sources.file.AbstractSource; -import org.apache.inlong.agent.plugin.sources.file.extend.DefaultExtendedHandler; import io.debezium.connector.sqlserver.SqlServerConnector; import io.debezium.connector.sqlserver.SqlServerConnectorConfig; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/extend/DefaultExtendedHandler.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/extend/DefaultExtendedHandler.java similarity index 95% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/extend/DefaultExtendedHandler.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/extend/DefaultExtendedHandler.java index 18df71bf31a..ce1e1182457 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/extend/DefaultExtendedHandler.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/extend/DefaultExtendedHandler.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.sources.file.extend; +package org.apache.inlong.agent.plugin.sources.extend; import org.apache.inlong.agent.conf.InstanceProfile; import org.apache.inlong.agent.plugin.Message; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/extend/ExtendedHandler.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/extend/ExtendedHandler.java similarity index 95% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/extend/ExtendedHandler.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/extend/ExtendedHandler.java index 2412a3055cf..9caf0826e3d 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/extend/ExtendedHandler.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/extend/ExtendedHandler.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.sources.file.extend; +package org.apache.inlong.agent.plugin.sources.extend; import org.apache.inlong.agent.conf.InstanceProfile; import org.apache.inlong.agent.plugin.Message; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/AbstractSource.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/AbstractSource.java index 8929b33d019..72d14327a2c 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/AbstractSource.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/file/AbstractSource.java @@ -29,7 +29,7 @@ import org.apache.inlong.agent.metrics.audit.AuditUtils; import org.apache.inlong.agent.plugin.Message; import org.apache.inlong.agent.plugin.file.Source; -import org.apache.inlong.agent.plugin.sources.file.extend.ExtendedHandler; +import org.apache.inlong.agent.plugin.sources.extend.ExtendedHandler; import org.apache.inlong.agent.utils.AgentUtils; import org.apache.inlong.agent.utils.ThreadUtils; import org.apache.inlong.common.metric.MetricRegister; @@ -80,8 +80,8 @@ protected class SourceData { protected final Integer BATCH_READ_LINE_COUNT = 10000; protected final Integer BATCH_READ_LINE_TOTAL_LEN = 1024 * 1024; protected final Integer CACHE_QUEUE_SIZE = 10 * BATCH_READ_LINE_COUNT; - protected final Integer READ_WAIT_TIMEOUT_MS = 10; - private final Integer EMPTY_CHECK_COUNT_AT_LEAST = 5 * 60; + protected final Integer WAIT_TIMEOUT_MS = 10; + private final Integer EMPTY_CHECK_COUNT_AT_LEAST = 5 * 60 * 100; private final Integer CORE_THREAD_PRINT_INTERVAL_MS = 1000; protected BlockingQueue queue; @@ -165,25 +165,23 @@ private void doRun() { break; } List lines = readFromSource(); - if (lines != null && lines.isEmpty()) { + if (lines == null || lines.isEmpty()) { if (queue.isEmpty()) { emptyCount++; } else { emptyCount = 0; } MemoryManager.getInstance().release(AGENT_GLOBAL_READER_SOURCE_PERMIT, BATCH_READ_LINE_TOTAL_LEN); - AgentUtils.silenceSleepInSeconds(1); + AgentUtils.silenceSleepInMs(WAIT_TIMEOUT_MS); continue; } emptyCount = 0; - if (lines != null) { - for (int i = 0; i < lines.size(); i++) { - boolean suc4Queue = waitForPermit(AGENT_GLOBAL_READER_QUEUE_PERMIT, lines.get(i).getData().length); - if (!suc4Queue) { - break; - } - putIntoQueue(lines.get(i)); + for (int i = 0; i < lines.size(); i++) { + boolean suc4Queue = waitForPermit(AGENT_GLOBAL_READER_QUEUE_PERMIT, lines.get(i).getData().length); + if (!suc4Queue) { + break; } + putIntoQueue(lines.get(i)); } MemoryManager.getInstance().release(AGENT_GLOBAL_READER_SOURCE_PERMIT, BATCH_READ_LINE_TOTAL_LEN); if (AgentUtils.getCurrentTime() - lastPrintTime > CORE_THREAD_PRINT_INTERVAL_MS) { @@ -231,7 +229,7 @@ private boolean waitForPermit(String permitName, int permitLen) { if (!isRunnable()) { return false; } - AgentUtils.silenceSleepInSeconds(1); + AgentUtils.silenceSleepInMs(WAIT_TIMEOUT_MS); } } return true; @@ -247,7 +245,7 @@ private void putIntoQueue(SourceData sourceData) { try { boolean offerSuc = false; while (isRunnable() && !offerSuc) { - offerSuc = queue.offer(sourceData, 1, TimeUnit.SECONDS); + offerSuc = queue.offer(sourceData, WAIT_TIMEOUT_MS, TimeUnit.MILLISECONDS); } if (!offerSuc) { MemoryManager.getInstance().release(AGENT_GLOBAL_READER_QUEUE_PERMIT, sourceData.getData().length); @@ -338,7 +336,7 @@ private boolean filterSourceData(Message msg) { private SourceData readFromQueue() { SourceData sourceData = null; try { - sourceData = queue.poll(READ_WAIT_TIMEOUT_MS, TimeUnit.MILLISECONDS); + sourceData = queue.poll(WAIT_TIMEOUT_MS, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { LOGGER.warn("poll {} data get interrupted.", instanceId); } @@ -405,7 +403,7 @@ private void clearQueue(BlockingQueue queue) { while (queue != null && !queue.isEmpty()) { SourceData sourceData = null; try { - sourceData = queue.poll(READ_WAIT_TIMEOUT_MS, TimeUnit.MILLISECONDS); + sourceData = queue.poll(WAIT_TIMEOUT_MS, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { LOGGER.warn("poll {} data get interrupted.", instanceId, e); } diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/KafkaTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/KafkaTask.java index 86b06079ab8..79d5534e5ca 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/KafkaTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/KafkaTask.java @@ -35,7 +35,6 @@ public class KafkaTask extends AbstractTask { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaTask.class); - public static final String DEFAULT_KAFKA_INSTANCE = "org.apache.inlong.agent.plugin.instance.KafkaInstance"; private boolean isAdded = false; private String topic; private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyyMMddHH"); @@ -58,8 +57,8 @@ protected List getNewInstanceList() { return list; } String dataTime = LocalDateTime.now().format(dateTimeFormatter); - InstanceProfile instanceProfile = taskProfile.createInstanceProfile(DEFAULT_KAFKA_INSTANCE, topic, - CycleUnitType.HOUR, dataTime, AgentUtils.getCurrentTime()); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(topic, CycleUnitType.HOUR, dataTime, + AgentUtils.getCurrentTime()); LOGGER.info("taskProfile.createInstanceProfile: {}", instanceProfile.toJsonStr()); list.add(instanceProfile); this.isAdded = true; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/MongoDBTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/MongoDBTask.java index 4a3ef07e269..bc1bcdc6491 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/MongoDBTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/MongoDBTask.java @@ -34,7 +34,6 @@ public class MongoDBTask extends AbstractTask { private static final Logger LOGGER = LoggerFactory.getLogger(MongoDBTask.class); - public static final String DEFAULT_MONGODB_INSTANCE = "org.apache.inlong.agent.plugin.instance.MongoDBInstance"; private boolean isAdded = false; private String collection; private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyyMMddHH"); @@ -66,8 +65,8 @@ protected List getNewInstanceList() { return list; } String dataTime = LocalDateTime.now().format(dateTimeFormatter); - InstanceProfile instanceProfile = taskProfile.createInstanceProfile(DEFAULT_MONGODB_INSTANCE, collection, - CycleUnitType.HOUR, dataTime, AgentUtils.getCurrentTime()); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(collection, CycleUnitType.HOUR, dataTime, + AgentUtils.getCurrentTime()); LOGGER.info("taskProfile.createInstanceProfile: {}", instanceProfile.toJsonStr()); list.add(instanceProfile); this.isAdded = true; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/MqttTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/MqttTask.java index 1d7d9a3dc2f..50de90a3157 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/MqttTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/MqttTask.java @@ -42,8 +42,6 @@ public class MqttTask extends AbstractTask { private AtomicBoolean isAdded = new AtomicBoolean(false); - public static final String DEFAULT_MQTT_INSTANCE = "org.apache.inlong.agent.plugin.instance.MqttInstance"; - private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyyMMddHH"); @Override @@ -93,8 +91,8 @@ protected List getNewInstanceList() { return list; } String dataTime = LocalDateTime.now().format(dateTimeFormatter); - InstanceProfile instanceProfile = taskProfile.createInstanceProfile(DEFAULT_MQTT_INSTANCE, topic, - CycleUnitType.HOUR, dataTime, AgentUtils.getCurrentTime()); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(topic, CycleUnitType.HOUR, dataTime, + AgentUtils.getCurrentTime()); LOGGER.info("taskProfile.createInstanceProfile(mqtt): {}", instanceProfile.toJsonStr()); list.add(instanceProfile); isAdded.set(true); diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/OracleTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/OracleTask.java index 34b064d48d5..4db4c0da211 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/OracleTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/OracleTask.java @@ -32,13 +32,13 @@ import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import static org.apache.inlong.agent.constant.TaskConstants.*; +import static org.apache.inlong.agent.constant.TaskConstants.TASK_ORACLE_DBNAME; +import static org.apache.inlong.agent.constant.TaskConstants.TASK_ORACLE_TABLE_INCLUDE_LIST; public class OracleTask extends AbstractTask { private static final Logger LOGGER = LoggerFactory.getLogger(OracleTask.class); - public static final String DEFAULT_ORACLE_INSTANCE = "org.apache.inlong.agent.plugin.instance.OracleInstance"; private AtomicBoolean isAdded = new AtomicBoolean(false); private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyyMMddHH"); @@ -103,9 +103,8 @@ protected List getNewInstanceList() { return list; } String dataTime = LocalDateTime.now().format(dateTimeFormatter); - InstanceProfile instanceProfile = - taskProfile.createInstanceProfile(DEFAULT_ORACLE_INSTANCE, instanceId, - CycleUnitType.HOUR, dataTime, AgentUtils.getCurrentTime()); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(instanceId, CycleUnitType.HOUR, dataTime, + AgentUtils.getCurrentTime()); LOGGER.info("taskProfile.createInstanceProfile: {}", instanceProfile.toJsonStr()); list.add(instanceProfile); this.isAdded.set(true); diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/PostgreSQLTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/PostgreSQLTask.java index 7cf382fbd63..db675c7e9fb 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/PostgreSQLTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/PostgreSQLTask.java @@ -41,7 +41,6 @@ public class PostgreSQLTask extends AbstractTask { private static final Logger LOGGER = LoggerFactory.getLogger(PostgreSQLTask.class); - public static final String DEFAULT_KAFKA_INSTANCE = "org.apache.inlong.agent.plugin.instance.KafkaInstance"; private final DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyyMMddHH"); private boolean isAdded = false; public static final int DEFAULT_INSTANCE_LIMIT = 1; @@ -98,8 +97,8 @@ protected List getNewInstanceList() { return list; } String dataTime = LocalDateTime.now().format(dateTimeFormatter); - InstanceProfile instanceProfile = taskProfile.createInstanceProfile(DEFAULT_KAFKA_INSTANCE, instanceId, - CycleUnitType.HOUR, dataTime, AgentUtils.getCurrentTime()); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(instanceId, CycleUnitType.HOUR, dataTime, + AgentUtils.getCurrentTime()); list.add(instanceProfile); this.isAdded = true; return list; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/PulsarTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/PulsarTask.java index 4586da7bb16..89da06bd1b8 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/PulsarTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/PulsarTask.java @@ -37,7 +37,6 @@ public class PulsarTask extends AbstractTask { private static final Logger LOGGER = LoggerFactory.getLogger(PulsarTask.class); - public static final String DEFAULT_PULSAR_INSTANCE = "org.apache.inlong.agent.plugin.instance.PulsarInstance"; private boolean isAdded = false; private String tenant; private String namespace; @@ -75,8 +74,8 @@ protected List getNewInstanceList() { return list; } String dataTime = LocalDateTime.now().format(dateTimeFormatter); - InstanceProfile instanceProfile = taskProfile.createInstanceProfile(DEFAULT_PULSAR_INSTANCE, instanceId, - CycleUnitType.HOUR, dataTime, AgentUtils.getCurrentTime()); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(instanceId, CycleUnitType.HOUR, dataTime, + AgentUtils.getCurrentTime()); LOGGER.info("taskProfile.createInstanceProfile: {}", instanceProfile.toJsonStr()); list.add(instanceProfile); this.isAdded = true; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/RedisTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/RedisTask.java index b9f7449ecd6..e36d5b02ee4 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/RedisTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/RedisTask.java @@ -34,7 +34,6 @@ public class RedisTask extends AbstractTask { private static final Logger LOGGER = LoggerFactory.getLogger(RedisTask.class); - public static final String DEFAULT_REDIS_INSTANCE = "org.apache.inlong.agent.plugin.instance.RedisInstance"; private boolean isAdded = false; private String taskId; @@ -68,8 +67,8 @@ protected List getNewInstanceList() { return list; } String dataTime = LocalDateTime.now().format(dateTimeFormatter); - InstanceProfile instanceProfile = taskProfile.createInstanceProfile(DEFAULT_REDIS_INSTANCE, taskId, - CycleUnitType.HOUR, dataTime, AgentUtils.getCurrentTime()); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(taskId, CycleUnitType.HOUR, dataTime, + AgentUtils.getCurrentTime()); LOGGER.info("taskProfile.createInstanceProfile: {}", instanceProfile.toJsonStr()); list.add(instanceProfile); this.isAdded = true; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/SQLServerTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/SQLServerTask.java index dd1446b301e..f892c80e338 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/SQLServerTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/SQLServerTask.java @@ -34,7 +34,6 @@ public class SQLServerTask extends AbstractTask { private static final Logger LOGGER = LoggerFactory.getLogger(SQLServerTask.class); - public static final String DEFAULT_SQLSERVER_INSTANCE = "org.apache.inlong.agent.plugin.instance.SQLServerInstance"; private boolean isAdded = false; private String dbName; @@ -102,8 +101,8 @@ protected List getNewInstanceList() { return list; } String dataTime = LocalDateTime.now().format(dateTimeFormatter); - InstanceProfile instanceProfile = taskProfile.createInstanceProfile(DEFAULT_SQLSERVER_INSTANCE, instanceId, - CycleUnitType.HOUR, dataTime, AgentUtils.getCurrentTime()); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(instanceId, CycleUnitType.HOUR, dataTime, + AgentUtils.getCurrentTime()); list.add(instanceProfile); this.isAdded = true; return list; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/AgentErrMsg.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/AgentErrMsg.java deleted file mode 100644 index aa7e5c734fd..00000000000 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/AgentErrMsg.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.inlong.agent.plugin.task.file; - -public class AgentErrMsg { - - public static final String CONFIG_SUCCESS = "SUCCESS"; - - // data source config error */ - public static final String DATA_SOURCE_CONFIG_ERROR = "ERROR-0-INLONG_AGENT|10001|ERROR" - + "|ERROR_DATA_SOURCE_CONFIG|"; - - // directory not found error */ - public static final String DIRECTORY_NOT_FOUND_ERROR = "ERROR-0-INLONG_AGENT|11001|WARN" - + "|WARN_DIRECTORY_NOT_EXIST|"; - - // watch directory error */ - public static final String WATCH_DIR_ERROR = "ERROR-0-INLONG_AGENT|11002|ERROR" - + "|ERROR_WATCH_DIR_ERROR|"; - - // file error(not found,rotate) - public static final String FILE_ERROR = "ERROR-0-INLONG_AGENT|10002|ERROR|ERROR_SOURCE_FILE|"; - - // read file error - public static final String FILE_OP_ERROR = "ERROR-1-INLONG_AGENT|30002|ERROR|ERROR_OPERATE_FILE|"; - - // disk full - public static final String DISK_FULL = "ERROR-1-INLONG_AGENT|30001|FATAL|FATAL_DISK_FULL|"; - - // out of memory - public static final String OOM_ERROR = "ERROR-1-INLONG_AGENT|30001|FATAL|FATAL_OOM_ERROR|"; - - // watcher error - public static final String WATCHER_INVALID = "ERROR-1-INLONG_AGENT|40001|WARN|WARN_INVALID_WATCHER|"; - - // could not connect to manager - public static final String CONNECT_MANAGER_ERROR = "ERROR-1-INLONG_AGENT|30002|ERROR" - + "|ERROR_CANNOT_CONNECT_TO_MANAGER|"; - - // send data to dataProxy failed - public static final String SEND_TO_BUS_ERROR = "ERROR-1-INLONG_AGENT|30003|ERROR|ERROR_SEND_TO_BUS|"; - - // operate bdb error - public static final String BDB_ERROR = "ERROR-1-INLONG_AGENT|30003|ERROR|BDB_OPERATION_ERROR|"; - - // buffer full - public static final String MSG_BUFFER_FULL = "ERROR-1-INLONG_AGENT|40002|WARN|WARN_MSG_BUFFER_FULL|"; - - // found event invalid(task has been delete) - public static final String FOUND_EVENT_INVALID = "ERROR-1-INLONG_AGENT|30003|ERROR" - + "|FOUND_EVENT_INVALID|"; -} diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/FileScanner.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/FileScanner.java deleted file mode 100644 index e37b6deb896..00000000000 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/FileScanner.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.inlong.agent.plugin.task.file; - -import org.apache.inlong.agent.plugin.utils.file.FilePathUtil; -import org.apache.inlong.agent.plugin.utils.file.FileTimeComparator; -import org.apache.inlong.agent.plugin.utils.file.Files; -import org.apache.inlong.agent.plugin.utils.file.NewDateUtils; -import org.apache.inlong.agent.utils.DateTransUtils; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.util.ArrayList; -import java.util.Calendar; -import java.util.Collections; -import java.util.List; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_FILE_MAX_NUM; - -/* - * This class is mainly used for scanning log file that we want to read. We use this class at - * inlong_agent recover process, the do and redo tasks and the current log file access when we deploy a - * new data source. - */ -public class FileScanner { - - public static class BasicFileInfo { - - public String fileName; - public String dataTime; - - public BasicFileInfo(String fileName, String dataTime) { - this.fileName = fileName; - this.dataTime = dataTime; - } - - } - - private static final Logger logger = LoggerFactory.getLogger(FileScanner.class); - - public static List getDataTimeList(long startTime, long endTime, String cycleUnit, String timeOffset, - boolean isRetry) { - if (!isRetry) { - startTime += DateTransUtils.calcOffset(timeOffset); - endTime += DateTransUtils.calcOffset(timeOffset); - } - List dataTimeList = new ArrayList<>(); - List dateRegion = NewDateUtils.getDateRegion(startTime, endTime, cycleUnit); - for (Long time : dateRegion) { - String dataTime = DateTransUtils.millSecConvertToTimeStr(time, cycleUnit); - dataTimeList.add(dataTime); - } - return dataTimeList; - } - - public static List scanTaskBetweenTimes(String originPattern, String cycleUnit, String timeOffset, - long startTime, long endTime, boolean isRetry) { - if (!isRetry) { - startTime += DateTransUtils.calcOffset(timeOffset); - endTime += DateTransUtils.calcOffset(timeOffset); - } - String strStartTime = DateTransUtils.millSecConvertToTimeStr(startTime, cycleUnit); - String strEndTime = DateTransUtils.millSecConvertToTimeStr(endTime, cycleUnit); - logger.info("{} scan time is between {} and {}", - new Object[]{originPattern, strStartTime, strEndTime}); - - return scanTaskBetweenTimes(cycleUnit, originPattern, startTime, endTime); - } - - /* Scan log files and create tasks between two times. */ - public static List scanTaskBetweenTimes(String cycleUnit, String originPattern, long startTime, - long endTime) { - List dateRegion = NewDateUtils.getDateRegion(startTime, endTime, cycleUnit); - List infos = new ArrayList(); - for (Long time : dateRegion) { - Calendar calendar = Calendar.getInstance(); - calendar.setTimeInMillis(time); - String fileName = NewDateUtils.replaceDateExpression(calendar, originPattern); - ArrayList allPaths = FilePathUtil.cutDirectoryByWildcard(fileName); - String firstDir = allPaths.get(0); - String secondDir = allPaths.get(0) + File.separator + allPaths.get(1); - ArrayList fileList = getUpdatedOrNewFiles(firstDir, secondDir, fileName, 3, - DEFAULT_FILE_MAX_NUM); - for (String file : fileList) { - // TODO the time is not YYYYMMDDHH - String dataTime = DateTransUtils.millSecConvertToTimeStr(time, cycleUnit); - BasicFileInfo info = new BasicFileInfo(file, dataTime); - logger.info("scan new task fileName {} ,dataTime {}", file, dataTime); - infos.add(info); - } - } - return infos; - } - - private static ArrayList getUpdatedOrNewFiles(String firstDir, String secondDir, - String fileName, long depth, int maxFileNum) { - ArrayList ret = new ArrayList(); - ArrayList readyFiles = new ArrayList(); - if (!new File(firstDir).isDirectory()) { - return ret; - } - for (File pathname : Files.find(firstDir).yieldFilesAndDirectories() - .recursive().withDepth((int) depth).withDirNameRegex(secondDir) - .withFileNameRegex(fileName)) { - if (readyFiles.size() >= maxFileNum) { - break; - } - readyFiles.add(pathname); - } - // sort by last-modified time (older -> newer) - Collections.sort(readyFiles, new FileTimeComparator()); - for (File f : readyFiles) { - // System.out.println(f.getAbsolutePath()); - ret.add(f.getAbsolutePath()); - } - return ret; - } - - @SuppressWarnings("unused") - private static ArrayList getUpdatedOrNewFiles(String logFileName, - int maxFileNum) { - ArrayList ret = new ArrayList(); - ArrayList directories = FilePathUtil - .cutDirectoryByWildcardAndDateExpression(logFileName); - String parentDir = directories.get(0) + File.separator - + directories.get(1); - - Pattern pattern = Pattern.compile(directories.get(2), - Pattern.CASE_INSENSITIVE); - for (File file : new File(parentDir).listFiles()) { - Matcher matcher = pattern.matcher(file.getName()); - if (matcher.matches() && ret.size() < maxFileNum) { - ret.add(file.getAbsolutePath()); - } - } - return ret; - } - - public static void main(String[] args) { - - ArrayList fileList = FileScanner.getUpdatedOrNewFiles( - "f:\\\\abc", "f:\\\\abc\\\\", "f:\\\\abc\\\\1.txt", 3, 100); - // fileList = FileScanner.getUpdatedOrNewFiles("F:\\abc\\1.txt", 100); - for (String fileName : fileList) { - System.out.println(fileName); - } - } -} diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/LogAbstractTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/LogAbstractTask.java new file mode 100644 index 00000000000..e2340425247 --- /dev/null +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/LogAbstractTask.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.agent.plugin.task.logcollection; + +import org.apache.inlong.agent.conf.InstanceProfile; +import org.apache.inlong.agent.core.task.TaskAction; +import org.apache.inlong.agent.plugin.task.AbstractTask; +import org.apache.inlong.agent.plugin.utils.regex.NewDateUtils; +import org.apache.inlong.agent.plugin.utils.regex.Scanner; +import org.apache.inlong.agent.state.State; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Date; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TimeZone; +import java.util.TreeSet; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingQueue; + +public abstract class LogAbstractTask extends AbstractTask { + + private static final int INSTANCE_QUEUE_CAPACITY = 10; + public static final long DAY_TIMEOUT_INTERVAL = 2 * 24 * 3600 * 1000; + private static final Logger LOGGER = LoggerFactory.getLogger(LogAbstractTask.class); + protected boolean retry; + protected BlockingQueue instanceQueue; + private volatile boolean runAtLeastOneTime = false; + protected long startTime; + protected long endTime; + protected String timeOffset = ""; + protected final Map> eventMap = + new ConcurrentHashMap<>(); + + @Override + protected void initTask() { + instanceQueue = new LinkedBlockingQueue<>(INSTANCE_QUEUE_CAPACITY); + } + + @Override + protected List getNewInstanceList() { + if (retry) { + runForRetry(); + } else { + runForNormal(); + } + List list = new ArrayList<>(); + while (list.size() < INSTANCE_QUEUE_CAPACITY && !instanceQueue.isEmpty()) { + InstanceProfile profile = instanceQueue.poll(); + if (profile != null) { + list.add(profile); + } + } + return list; + } + + abstract protected void runForNormal(); + + abstract protected void dealWithEventMap(); + + abstract protected void scanExistingFile(); + + private void runForRetry() { + if (!runAtLeastOneTime) { + scanExistingFile(); + runAtLeastOneTime = true; + } + dealWithEventMap(); + if (allInstanceFinished()) { + LOGGER.info("retry task finished, send action to task manager, taskId {}", getTaskId()); + TaskAction action = new TaskAction(org.apache.inlong.agent.core.task.ActionType.FINISH, taskProfile); + taskManager.submitAction(action); + doChangeState(State.SUCCEEDED); + } + } + + protected void dealWithEventMapWithCycle() { + long startScanTime = startTime; + long endScanTime = endTime; + List dataTimeList = Scanner.getDataTimeList(startScanTime, endScanTime, taskProfile.getCycleUnit(), + timeOffset, retry); + if (dataTimeList.isEmpty()) { + LOGGER.error("getDataTimeList get empty list"); + return; + } + Set dealtDataTime = new HashSet<>(); + // normal task first handle current data time + if (!retry) { + String current = dataTimeList.remove(dataTimeList.size() - 1); + dealtDataTime.add(current); + if (!dealEventMapByDataTime(current, true)) { + return; + } + } + dataTimeList.forEach(dataTime -> { + dealtDataTime.add(dataTime); + if (!dealEventMapByDataTime(dataTime, false)) { + return; + } + }); + for (String dataTime : eventMap.keySet()) { + if (!dealtDataTime.contains(dataTime)) { + dealEventMapByDataTime(dataTime, false); + } + } + } + + protected boolean dealEventMapByDataTime(String dataTime, boolean isCurrentDataTime) { + Map sameDataTimeEvents = eventMap.get(dataTime); + if (sameDataTimeEvents == null || sameDataTimeEvents.isEmpty()) { + return true; + } + if (shouldStartNow(dataTime)) { + Set sortedEvents = new TreeSet<>(Comparator.comparing(InstanceProfile::getInstanceId)); + sortedEvents.addAll(sameDataTimeEvents.values()); + for (InstanceProfile sortEvent : sortedEvents) { + String fileName = sortEvent.getInstanceId(); + InstanceProfile profile = sameDataTimeEvents.get(fileName); + if (!isCurrentDataTime && isFull()) { + return false; + } + if (!instanceQueue.offer(profile)) { + return false; + } + sameDataTimeEvents.remove(fileName); + } + } + return true; + } + + /* + * Calculate whether the event needs to be processed at the current time based on its data time, business cycle, and + * offset + */ + private boolean shouldStartNow(String dataTime) { + String shouldStartTime = NewDateUtils.getShouldStartTime(dataTime, taskProfile.getCycleUnit(), timeOffset); + String currentTime = getCurrentTime(); + return currentTime.compareTo(shouldStartTime) >= 0; + } + + private String getCurrentTime() { + SimpleDateFormat dateFormat = new SimpleDateFormat(NewDateUtils.DEFAULT_FORMAT); + TimeZone timeZone = TimeZone.getTimeZone(NewDateUtils.DEFAULT_TIME_ZONE); + dateFormat.setTimeZone(timeZone); + return dateFormat.format(new Date(System.currentTimeMillis())); + } + + protected void removeTimeoutEvent(Map> eventMap, boolean isRetry) { + if (isRetry) { + return; + } + for (Map.Entry> entry : eventMap.entrySet()) { + /* If the data time of the event is within 2 days before (after) the current time, it is valid */ + String dataTime = entry.getKey(); + if (!NewDateUtils.isValidCreationTime(dataTime, DAY_TIMEOUT_INTERVAL)) { + /* Remove it from memory map. */ + eventMap.remove(dataTime); + LOGGER.warn("remove too old event from event map. dataTime {}", dataTime); + } + } + } +} diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/cos/COSTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/cos/COSTask.java new file mode 100644 index 00000000000..86210157ed6 --- /dev/null +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/cos/COSTask.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.agent.plugin.task.logcollection.cos; + +import org.apache.inlong.agent.conf.InstanceProfile; +import org.apache.inlong.agent.conf.TaskProfile; +import org.apache.inlong.agent.constant.TaskConstants; +import org.apache.inlong.agent.plugin.task.logcollection.LogAbstractTask; +import org.apache.inlong.agent.plugin.task.logcollection.cos.FileScanner.BasicFileInfo; +import org.apache.inlong.agent.plugin.utils.cos.COSUtils; +import org.apache.inlong.agent.utils.AgentUtils; +import org.apache.inlong.agent.utils.DateTransUtils; + +import com.qcloud.cos.COSClient; +import com.qcloud.cos.model.ObjectMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.text.ParseException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Watch directory, if new valid files are created, create instance correspondingly. + */ +public class COSTask extends LogAbstractTask { + + private static final Logger LOGGER = LoggerFactory.getLogger(COSTask.class); + private String originPattern; + private long lastScanTime = 0; + public final long SCAN_INTERVAL = 1 * 60 * 1000; + private COSClient cosClient; + private String bucketName; + private String secretId; + private String secretKey; + private String strRegion; + + @Override + protected int getInstanceLimit() { + return taskProfile.getInt(TaskConstants.COS_MAX_NUM); + } + + @Override + protected void initTask() { + super.initTask(); + timeOffset = taskProfile.get(TaskConstants.TASK_COS_TIME_OFFSET, ""); + retry = taskProfile.getBoolean(TaskConstants.COS_TASK_RETRY, false); + originPattern = taskProfile.get(TaskConstants.COS_TASK_PATTERN); + bucketName = taskProfile.get(TaskConstants.COS_TASK_BUCKET_NAME); + secretId = taskProfile.get(TaskConstants.COS_TASK_SECRET_ID); + secretKey = taskProfile.get(TaskConstants.COS_TASK_SECRET_KEY); + strRegion = taskProfile.get(TaskConstants.COS_TASK_REGION); + cosClient = COSUtils.createCli(secretId, secretKey, strRegion); + if (retry) { + initRetryTask(taskProfile); + } + } + + private boolean initRetryTask(TaskProfile profile) { + String dataTimeFrom = profile.get(TaskConstants.COS_TASK_TIME_FROM, ""); + String dataTimeTo = profile.get(TaskConstants.COS_TASK_TIME_TO, ""); + try { + startTime = DateTransUtils.timeStrConvertToMillSec(dataTimeFrom, profile.getCycleUnit()); + endTime = DateTransUtils.timeStrConvertToMillSec(dataTimeTo, profile.getCycleUnit()); + } catch (ParseException e) { + LOGGER.error("retry task time error start {} end {}", dataTimeFrom, dataTimeTo, e); + return false; + } + return true; + } + + @Override + public boolean isProfileValid(TaskProfile profile) { + if (!profile.allRequiredKeyExist()) { + LOGGER.error("task profile needs all required key"); + return false; + } + if (!profile.hasKey(TaskConstants.COS_TASK_CYCLE_UNIT)) { + LOGGER.error("task profile needs cos cycle unit"); + return false; + } + if (!profile.hasKey(TaskConstants.TASK_CYCLE_UNIT)) { + LOGGER.error("task profile needs cycle unit"); + return false; + } + if (profile.get(TaskConstants.TASK_CYCLE_UNIT) + .compareTo(profile.get(TaskConstants.COS_TASK_CYCLE_UNIT)) != 0) { + LOGGER.error("task profile cycle unit must be consistent"); + return false; + } + if (!profile.hasKey(TaskConstants.TASK_TIME_ZONE)) { + LOGGER.error("task profile needs time zone"); + return false; + } + boolean ret = profile.hasKey(TaskConstants.COS_TASK_PATTERN) + && profile.hasKey(TaskConstants.COS_MAX_NUM); + if (!ret) { + LOGGER.error("task profile needs file keys"); + return false; + } + if (!profile.hasKey(TaskConstants.TASK_COS_TIME_OFFSET)) { + LOGGER.error("task profile needs time offset"); + return false; + } + if (profile.getBoolean(TaskConstants.COS_TASK_RETRY, false)) { + if (!initRetryTask(profile)) { + return false; + } + } + return true; + } + + @Override + protected void releaseTask() { + cosClient.shutdown(); + } + + @Override + protected void runForNormal() { + if (AgentUtils.getCurrentTime() - lastScanTime > SCAN_INTERVAL) { + scanExistingFile(); + lastScanTime = AgentUtils.getCurrentTime(); + } + dealWithEventMap(); + } + + @Override + protected void scanExistingFile() { + List fileInfos = FileScanner.scanTaskBetweenTimes(cosClient, bucketName, originPattern, + taskProfile.getCycleUnit(), timeOffset, startTime, endTime, retry); + LOGGER.info("taskId {} scan {} get file count {}", getTaskId(), originPattern, fileInfos.size()); + fileInfos.forEach((fileInfo) -> { + addToEvenMap(fileInfo.fileName, fileInfo.dataTime); + if (retry) { + instanceCount++; + } + }); + } + + private boolean isInEventMap(String fileName, String dataTime) { + Map fileToProfile = eventMap.get(dataTime); + if (fileToProfile == null) { + return false; + } + return fileToProfile.get(fileName) != null; + } + + @Override + protected void dealWithEventMap() { + removeTimeoutEvent(eventMap, retry); + dealWithEventMapWithCycle(); + } + + private void addToEvenMap(String fileName, String dataTime) { + if (isInEventMap(fileName, dataTime)) { + LOGGER.info("add to evenMap isInEventMap returns true skip taskId {} dataTime {} fileName {}", + taskProfile.getTaskId(), dataTime, fileName); + return; + } + ObjectMetadata meta = cosClient.getObjectMetadata(bucketName, fileName); + Long fileUpdateTime = meta.getLastModified().getTime(); + if (!shouldAddAgain(fileName, fileUpdateTime)) { + LOGGER.info("add to evenMap shouldAddAgain returns false skip taskId {} dataTime {} fileName {}", + taskProfile.getTaskId(), dataTime, fileName); + return; + } + Map sameDataTimeEvents = eventMap.computeIfAbsent(dataTime, + mapKey -> new ConcurrentHashMap<>()); + boolean containsInMemory = sameDataTimeEvents.containsKey(fileName); + if (containsInMemory) { + LOGGER.error("should not happen! may be {} has been deleted and add again", fileName); + return; + } + String cycleUnit = taskProfile.getCycleUnit(); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(fileName, cycleUnit, dataTime, + fileUpdateTime); + sameDataTimeEvents.put(fileName, instanceProfile); + LOGGER.info("add to eventMap taskId {} dataTime {} fileName {}", taskProfile.getTaskId(), dataTime, fileName); + } +} diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/cos/FileScanner.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/cos/FileScanner.java new file mode 100644 index 00000000000..e3185722dd5 --- /dev/null +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/cos/FileScanner.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.agent.plugin.task.logcollection.cos; + +import org.apache.inlong.agent.plugin.utils.regex.PatternUtil; +import org.apache.inlong.agent.plugin.utils.regex.Scanner; +import org.apache.inlong.agent.plugin.utils.regex.Scanner.FinalPatternInfo; +import org.apache.inlong.agent.utils.DateTransUtils; + +import com.qcloud.cos.COSClient; +import com.qcloud.cos.exception.CosClientException; +import com.qcloud.cos.exception.CosServiceException; +import com.qcloud.cos.model.COSObjectSummary; +import com.qcloud.cos.model.ListObjectsRequest; +import com.qcloud.cos.model.ObjectListing; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/* + * This class is mainly used for scanning log file that we want to read. We use this class at + * inlong_agent recover process, the do and redo tasks and the current log file access when we deploy a + * new data source. + */ +public class FileScanner { + + public static final int DEFAULT_KEY_COUNT = 100; + public static final String DEFAULT_DELIMITER = "/"; + public static final char PATH_SEP = '/'; + + public static class BasicFileInfo { + + public String fileName; + public String dataTime; + + public BasicFileInfo(String fileName, String dataTime) { + this.fileName = fileName; + this.dataTime = dataTime; + } + } + + private static final Logger logger = LoggerFactory.getLogger(FileScanner.class); + + public static List scanTaskBetweenTimes(COSClient cosClient, String bucketName, String originPattern, + String cycleUnit, String timeOffset, long startTime, long endTime, boolean isRetry) { + List finalPatternInfos = Scanner.getFinalPatternInfos(originPattern, cycleUnit, timeOffset, + startTime, endTime, isRetry); + List infos = new ArrayList<>(); + for (FinalPatternInfo finalPatternInfo : finalPatternInfos) { + String prefix = PatternUtil.getBeforeFirstWildcard(finalPatternInfo.finalPattern); + Pattern pattern = Pattern.compile(finalPatternInfo.finalPattern, + Pattern.CASE_INSENSITIVE | Pattern.DOTALL | Pattern.MULTILINE); + List fileInfos = scanTaskInOneCycle(cosClient, bucketName, pattern, prefix, + finalPatternInfo.dataTime, cycleUnit); + infos.addAll(fileInfos); + } + return infos; + } + + public static List scanTaskInOneCycle(COSClient cosClient, String bucketName, Pattern pattern, + String prefix, Long dataTime, String cycleUnit) { + List infos = new ArrayList<>(); + ObjectListing objectListing; + ListObjectsRequest listObjectsRequest = new ListObjectsRequest(); + do { + try { + listObjectsRequest.setBucketName(bucketName); + listObjectsRequest.setPrefix(prefix); + listObjectsRequest.setDelimiter(DEFAULT_DELIMITER); + listObjectsRequest.setMaxKeys(DEFAULT_KEY_COUNT); + objectListing = cosClient.listObjects(listObjectsRequest); + } catch (CosServiceException e) { + logger.error("scanTaskInOneCycle finalPattern {} CosServiceException", pattern.pattern(), e); + return infos; + } catch (CosClientException e) { + logger.error("scanTaskInOneCycle finalPattern {} CosClientException", pattern.pattern(), e); + return infos; + } + List commonPrefixes = objectListing.getCommonPrefixes(); + int depth; + Pattern patternByDepth; + if (!commonPrefixes.isEmpty()) { + depth = countCharacterOccurrences(commonPrefixes.get(0), PATH_SEP); + String nthOccurrenceSubstring = findNthOccurrenceSubstring(pattern.pattern(), PATH_SEP, depth); + if (nthOccurrenceSubstring != null) { + patternByDepth = Pattern.compile(nthOccurrenceSubstring, + Pattern.CASE_INSENSITIVE | Pattern.DOTALL | Pattern.MULTILINE); + for (String commonPrefix : commonPrefixes) { + Matcher matcher = patternByDepth.matcher(commonPrefix); + if (matcher.matches()) { + infos.addAll(scanTaskInOneCycle(cosClient, bucketName, pattern, commonPrefix, dataTime, + cycleUnit)); + } + } + } + } + List cosObjectSummaries = objectListing.getObjectSummaries(); + for (COSObjectSummary cosObjectSummary : cosObjectSummaries) { + String key = cosObjectSummary.getKey(); + Matcher matcher = pattern.matcher(key); + if (matcher.lookingAt()) { + long fileSize = cosObjectSummary.getSize(); + String storageClasses = cosObjectSummary.getStorageClass(); + infos.add(new BasicFileInfo(key, + DateTransUtils.millSecConvertToTimeStr(dataTime, cycleUnit))); + String strDataTime = DateTransUtils.millSecConvertToTimeStr(dataTime, cycleUnit); + logger.info("list key {} dataTime {} size {}, storageClasses {}", key, strDataTime, fileSize, + storageClasses); + } + } + String nextMarker = objectListing.getNextMarker(); + listObjectsRequest.setMarker(nextMarker); + } while (objectListing.isTruncated()); + return infos; + } + + public static int countCharacterOccurrences(String input, char targetChar) { + if (input == null) { + throw new IllegalArgumentException("Input string cannot be null"); + } + int count = 0; + + for (int i = 0; i < input.length(); i++) { + if (input.charAt(i) == targetChar) { + count++; + } + } + return count; + } + + public static String findNthOccurrenceSubstring(String input, char targetChar, int n) { + int endIndex = findNthOccurrence(input, targetChar, n); + if (endIndex != -1) { + return input.substring(0, endIndex + 1); + } else { + return null; + } + } + + public static int findNthOccurrence(String input, char targetChar, int n) { + int currentIndex = -1; + for (int i = 0; i < n; i++) { + currentIndex = input.indexOf(targetChar, currentIndex + 1); + if (currentIndex == -1) { + return -1; + } + } + return currentIndex; + } +} diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/FileDataUtils.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileDataUtils.java similarity index 95% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/FileDataUtils.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileDataUtils.java index 57b4702848d..5848ad563d6 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/FileDataUtils.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileDataUtils.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.utils.file; +package org.apache.inlong.agent.plugin.task.logcollection.local; import java.io.IOException; import java.nio.file.Files; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileScanner.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileScanner.java new file mode 100644 index 00000000000..70003d65ab4 --- /dev/null +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileScanner.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.agent.plugin.task.logcollection.local; + +import org.apache.inlong.agent.plugin.utils.regex.PatternUtil; +import org.apache.inlong.agent.plugin.utils.regex.Scanner; +import org.apache.inlong.agent.plugin.utils.regex.Scanner.FinalPatternInfo; +import org.apache.inlong.agent.utils.DateTransUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.inlong.agent.constant.CommonConstants.DEFAULT_FILE_MAX_NUM; + +/* + * This class is mainly used for scanning log file that we want to read. We use this class at + * inlong_agent recover process, the do and redo tasks and the current log file access when we deploy a + * new data source. + */ +public class FileScanner { + + public static class BasicFileInfo { + + public String fileName; + public String dataTime; + + public BasicFileInfo(String fileName, String dataTime) { + this.fileName = fileName; + this.dataTime = dataTime; + } + } + + private static final Logger logger = LoggerFactory.getLogger(FileScanner.class); + + public static List scanTaskBetweenTimes(String originPattern, String cycleUnit, String timeOffset, + long startTime, long endTime, boolean isRetry) { + List infos = new ArrayList<>(); + List finalPatternInfos = Scanner.getFinalPatternInfos(originPattern, cycleUnit, timeOffset, + startTime, endTime, isRetry); + for (FinalPatternInfo finalPatternInfo : finalPatternInfos) { + ArrayList allPaths = PatternUtil.cutDirectoryByWildcard(finalPatternInfo.finalPattern); + String firstDir = allPaths.get(0); + String secondDir = allPaths.get(0) + File.separator + allPaths.get(1); + ArrayList fileList = getUpdatedOrNewFiles(firstDir, secondDir, finalPatternInfo.finalPattern, 3, + DEFAULT_FILE_MAX_NUM); + for (String file : fileList) { + // TODO the time is not YYYYMMDDHH + String dataTime = DateTransUtils.millSecConvertToTimeStr(finalPatternInfo.dataTime, cycleUnit); + BasicFileInfo info = new BasicFileInfo(file, dataTime); + logger.info("scan new task fileName {} ,dataTime {}", file, dataTime); + infos.add(info); + } + } + return infos; + } + + private static ArrayList getUpdatedOrNewFiles(String firstDir, String secondDir, + String fileName, long depth, int maxFileNum) { + ArrayList ret = new ArrayList(); + ArrayList readyFiles = new ArrayList(); + if (!new File(firstDir).isDirectory()) { + return ret; + } + for (File pathname : Files.find(firstDir).yieldFilesAndDirectories() + .recursive().withDepth((int) depth).withDirNameRegex(secondDir) + .withFileNameRegex(fileName)) { + if (readyFiles.size() >= maxFileNum) { + break; + } + readyFiles.add(pathname); + } + // sort by last-modified time (older -> newer) + Collections.sort(readyFiles, new FileTimeComparator()); + for (File f : readyFiles) { + // System.out.println(f.getAbsolutePath()); + ret.add(f.getAbsolutePath()); + } + return ret; + } +} diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/LogFileTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileTask.java similarity index 64% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/LogFileTask.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileTask.java index 4f49cfdd7dd..ef50d1fc62a 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/LogFileTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileTask.java @@ -15,19 +15,17 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.task.file; +package org.apache.inlong.agent.plugin.task.logcollection.local; import org.apache.inlong.agent.conf.InstanceProfile; import org.apache.inlong.agent.conf.TaskProfile; import org.apache.inlong.agent.constant.CycleUnitType; import org.apache.inlong.agent.constant.TaskConstants; -import org.apache.inlong.agent.core.task.TaskAction; -import org.apache.inlong.agent.plugin.task.AbstractTask; -import org.apache.inlong.agent.plugin.task.file.FileScanner.BasicFileInfo; -import org.apache.inlong.agent.plugin.utils.file.FilePathUtil; -import org.apache.inlong.agent.plugin.utils.file.NewDateUtils; -import org.apache.inlong.agent.plugin.utils.file.PathDateExpression; -import org.apache.inlong.agent.state.State; +import org.apache.inlong.agent.plugin.task.logcollection.LogAbstractTask; +import org.apache.inlong.agent.plugin.task.logcollection.local.FileScanner.BasicFileInfo; +import org.apache.inlong.agent.plugin.utils.regex.NewDateUtils; +import org.apache.inlong.agent.plugin.utils.regex.PathDateExpression; +import org.apache.inlong.agent.plugin.utils.regex.PatternUtil; import org.apache.inlong.agent.utils.AgentUtils; import org.apache.inlong.agent.utils.DateTransUtils; import org.apache.inlong.agent.utils.file.FileUtils; @@ -45,47 +43,31 @@ import java.nio.file.WatchEvent.Kind; import java.nio.file.WatchKey; import java.nio.file.WatchService; -import java.text.SimpleDateFormat; +import java.text.ParseException; import java.util.ArrayList; -import java.util.Date; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TimeZone; -import java.util.TreeSet; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.LinkedBlockingQueue; import java.util.regex.Matcher; import java.util.stream.Collectors; import java.util.stream.Stream; /** - * Watch directory, if new valid files are created, create jobs correspondingly. + * Watch directory, if new valid files are created, create instance correspondingly. */ -public class LogFileTask extends AbstractTask { +public class FileTask extends LogAbstractTask { - private static final Logger LOGGER = LoggerFactory.getLogger(LogFileTask.class); - public static final String DEFAULT_FILE_INSTANCE = "org.apache.inlong.agent.plugin.instance.FileInstance"; - public static final String SCAN_CYCLE_RANCE = "-2"; - private static final int INSTANCE_QUEUE_CAPACITY = 10; + private static final Logger LOGGER = LoggerFactory.getLogger(FileTask.class); private final Map watchers = new ConcurrentHashMap<>(); private final Set watchFailedDirs = new HashSet<>(); - private final Map> eventMap = - new ConcurrentHashMap<>(); - public static final long DAY_TIMEOUT_INTERVAL = 2 * 24 * 3600 * 1000; public static final int CORE_THREAD_MAX_GAP_TIME_MS = 60 * 1000; - private boolean retry; - private long startTime; - private long endTime; private boolean realTime = false; private Set originPatterns; private long lastScanTime = 0; public final long SCAN_INTERVAL = 1 * 60 * 1000; - private volatile boolean runAtLeastOneTime = false; private volatile long coreThreadUpdateTime = 0; - private BlockingQueue instanceQueue; @Override protected int getInstanceLimit() { @@ -94,35 +76,32 @@ protected int getInstanceLimit() { @Override protected void initTask() { - instanceQueue = new LinkedBlockingQueue<>(INSTANCE_QUEUE_CAPACITY); - retry = taskProfile.getBoolean(TaskConstants.TASK_RETRY, false); + super.initTask(); + timeOffset = taskProfile.get(TaskConstants.TASK_FILE_TIME_OFFSET, ""); + retry = taskProfile.isRetry(); originPatterns = Stream.of(taskProfile.get(TaskConstants.FILE_DIR_FILTER_PATTERNS).split(",")) .collect(Collectors.toSet()); if (taskProfile.getCycleUnit().compareToIgnoreCase(CycleUnitType.REAL_TIME) == 0) { realTime = true; } if (retry) { - retryInit(); + initRetryTask(taskProfile); } else { watchInit(); } } - @Override - protected List getNewInstanceList() { - if (retry) { - runForRetry(); - } else { - runForNormal(); - } - List list = new ArrayList<>(); - while (list.size() < INSTANCE_QUEUE_CAPACITY && !instanceQueue.isEmpty()) { - InstanceProfile profile = instanceQueue.poll(); - if (profile != null) { - list.add(profile); - } + private boolean initRetryTask(TaskProfile profile) { + String dataTimeFrom = profile.get(TaskConstants.FILE_TASK_TIME_FROM, ""); + String dataTimeTo = profile.get(TaskConstants.FILE_TASK_TIME_TO, ""); + try { + startTime = DateTransUtils.timeStrConvertToMillSec(dataTimeFrom, profile.getCycleUnit()); + endTime = DateTransUtils.timeStrConvertToMillSec(dataTimeTo, profile.getCycleUnit()); + } catch (ParseException e) { + LOGGER.error("retry task time error start {} end {}", dataTimeFrom, dataTimeTo, e); + return false; } - return list; + return true; } @Override @@ -159,22 +138,14 @@ public boolean isProfileValid(TaskProfile profile) { LOGGER.error("task profile needs time offset"); return false; } - if (profile.getBoolean(TaskConstants.TASK_RETRY, false)) { - long startTime = profile.getLong(TaskConstants.TASK_START_TIME, 0); - long endTime = profile.getLong(TaskConstants.TASK_END_TIME, 0); - if (startTime == 0 || endTime == 0) { - LOGGER.error("retry task time error start {} end {}", startTime, endTime); + if (profile.isRetry()) { + if (!initRetryTask(profile)) { return false; } } return true; } - private void retryInit() { - startTime = taskProfile.getLong(TaskConstants.TASK_START_TIME, 0); - endTime = taskProfile.getLong(TaskConstants.TASK_END_TIME, 0); - } - private void watchInit() { originPatterns.forEach((pathPattern) -> { addPathPattern(pathPattern); @@ -182,12 +153,12 @@ private void watchInit() { } public void addPathPattern(String originPattern) { - ArrayList directories = FilePathUtil.cutDirectoryByWildcardAndDateExpression(originPattern); + ArrayList directories = PatternUtil.cutDirectoryByWildcardAndDateExpression(originPattern); String basicStaticPath = directories.get(0); LOGGER.info("dataName {} watchPath {}", new Object[]{originPattern, basicStaticPath}); /* Remember the failed watcher creations. */ if (!new File(basicStaticPath).exists()) { - LOGGER.warn(AgentErrMsg.DIRECTORY_NOT_FOUND_ERROR + basicStaticPath); + LOGGER.warn("DIRECTORY_NOT_FOUND_ERROR" + basicStaticPath); watchFailedDirs.add(originPattern); return; } @@ -204,9 +175,9 @@ public void addPathPattern(String originPattern) { watchFailedDirs.remove(originPattern); } catch (IOException e) { if (e.toString().contains("Too many open files") || e.toString().contains("打开的文件过多")) { - LOGGER.error(AgentErrMsg.WATCH_DIR_ERROR + e.toString()); + LOGGER.error("WATCH_DIR_ERROR", e); } else { - LOGGER.error(AgentErrMsg.WATCH_DIR_ERROR + e.toString(), e); + LOGGER.error("WATCH_DIR_ERROR", e); } } catch (Exception e) { LOGGER.error("addPathPattern:", e); @@ -235,21 +206,8 @@ private void releaseWatchers(Map watchers) { }); } - private void runForRetry() { - if (!runAtLeastOneTime) { - scanExistingFile(); - runAtLeastOneTime = true; - } - dealWithEventMap(); - if (allInstanceFinished()) { - LOGGER.info("retry task finished, send action to task manager, taskId {}", getTaskId()); - TaskAction action = new TaskAction(org.apache.inlong.agent.core.task.ActionType.FINISH, taskProfile); - taskManager.submitAction(action); - doChangeState(State.SUCCEEDED); - } - } - - private void runForNormal() { + @Override + protected void runForNormal() { if (AgentUtils.getCurrentTime() - lastScanTime > SCAN_INTERVAL) { scanExistingFile(); lastScanTime = AgentUtils.getCurrentTime(); @@ -258,7 +216,8 @@ private void runForNormal() { dealWithEventMap(); } - private void scanExistingFile() { + @Override + protected void scanExistingFile() { originPatterns.forEach((originPattern) -> { List fileInfos = scanExistingFileByPattern(originPattern); LOGGER.info("taskId {} scan {} get file count {}", getTaskId(), originPattern, fileInfos.size()); @@ -283,21 +242,12 @@ private boolean isInEventMap(String fileName, String dataTime) { } private List scanExistingFileByPattern(String originPattern) { - long startScanTime = startTime; - long endScanTime = endTime; - if (!retry) { - long currentTime = System.currentTimeMillis(); - // only scan two cycle, like two hours or two days - long offset = DateTransUtils.calcOffset(SCAN_CYCLE_RANCE + taskProfile.getCycleUnit()); - startScanTime = currentTime + offset; - endScanTime = currentTime; - } if (realTime) { - return FileScanner.scanTaskBetweenTimes(originPattern, CycleUnitType.HOUR, taskProfile.getTimeOffset(), - startScanTime, endScanTime, retry); + return FileScanner.scanTaskBetweenTimes(originPattern, CycleUnitType.HOUR, timeOffset, + startTime, endTime, retry); } else { return FileScanner.scanTaskBetweenTimes(originPattern, taskProfile.getCycleUnit(), - taskProfile.getTimeOffset(), startScanTime, endScanTime, retry); + timeOffset, startTime, endTime, retry); } } @@ -316,7 +266,8 @@ private void runForWatching() { } } - private void dealWithEventMap() { + @Override + protected void dealWithEventMap() { removeTimeoutEvent(eventMap, retry); if (realTime) { dealWithEventMapRealTime(); @@ -325,102 +276,12 @@ private void dealWithEventMap() { } } - private void dealWithEventMapWithCycle() { - long startScanTime = startTime; - long endScanTime = endTime; - if (!retry) { - long currentTime = System.currentTimeMillis(); - // only scan two cycle, like two hours or two days - long offset = DateTransUtils.calcOffset(SCAN_CYCLE_RANCE + taskProfile.getCycleUnit()); - startScanTime = currentTime + offset; - endScanTime = currentTime; - } - List dataTimeList = FileScanner.getDataTimeList(startScanTime, endScanTime, taskProfile.getCycleUnit(), - taskProfile.getTimeOffset(), retry); - if (dataTimeList.isEmpty()) { - LOGGER.error("getDataTimeList get empty list"); - return; - } - Set dealtDataTime = new HashSet<>(); - // normal task first handle current data time - if (!retry) { - String current = dataTimeList.remove(dataTimeList.size() - 1); - dealEventMapByDataTime(current, true); - dealtDataTime.add(current); - } - dataTimeList.forEach(dataTime -> { - dealtDataTime.add(dataTime); - dealEventMapByDataTime(dataTime, false); - }); - for (String dataTime : eventMap.keySet()) { - if (!dealtDataTime.contains(dataTime)) { - dealEventMapByDataTime(dataTime, false); - } - } - } - private void dealWithEventMapRealTime() { for (String dataTime : eventMap.keySet()) { dealEventMapByDataTime(dataTime, true); } } - private void dealEventMapByDataTime(String dataTime, boolean isCurrentDataTime) { - Map sameDataTimeEvents = eventMap.get(dataTime); - if (sameDataTimeEvents == null || sameDataTimeEvents.isEmpty()) { - return; - } - if (realTime || shouldStartNow(dataTime)) { - /* These codes will sort the FileCreationEvents by create time. */ - Set sortedEvents = new TreeSet<>(sameDataTimeEvents.values()); - /* Check the file end with event creation time in asc order. */ - for (InstanceProfile sortEvent : sortedEvents) { - String fileName = sortEvent.getInstanceId(); - InstanceProfile profile = sameDataTimeEvents.get(fileName); - if (!isCurrentDataTime && isFull()) { - return; - } - if (!instanceQueue.offer(profile)) { - return; - } - sameDataTimeEvents.remove(fileName); - } - } - } - - /* - * Calculate whether the event needs to be processed at the current time based on its data time, business cycle, and - * offset - */ - private boolean shouldStartNow(String dataTime) { - String shouldStartTime = - NewDateUtils.getShouldStartTime(dataTime, taskProfile.getCycleUnit(), taskProfile.getTimeOffset()); - String currentTime = getCurrentTime(); - return currentTime.compareTo(shouldStartTime) >= 0; - } - - private void removeTimeoutEvent(Map> eventMap, boolean isRetry) { - if (isRetry || realTime) { - return; - } - for (Map.Entry> entry : eventMap.entrySet()) { - /* If the data time of the event is within 2 days before (after) the current time, it is valid */ - String dataTime = entry.getKey(); - if (!NewDateUtils.isValidCreationTime(dataTime, DAY_TIMEOUT_INTERVAL)) { - /* Remove it from memory map. */ - eventMap.remove(dataTime); - LOGGER.warn("remove too old event from event map. dataTime {}", dataTime); - } - } - } - - private String getCurrentTime() { - SimpleDateFormat dateFormat = new SimpleDateFormat(NewDateUtils.DEFAULT_FORMAT); - TimeZone timeZone = TimeZone.getTimeZone(NewDateUtils.DEFAULT_TIME_ZONE); - dateFormat.setTimeZone(timeZone); - return dateFormat.format(new Date(System.currentTimeMillis())); - } - public synchronized void dealWithWatchEntity(String originPattern) { WatchEntity entity = watchers.get(originPattern); if (entity == null) { @@ -484,14 +345,14 @@ private Path resolvePathFromEvent(WatchEvent watchEvent, Path contextPath) { private void handleFilePath(Path filePath, WatchEntity entity) { String newFileName = filePath.toFile().getAbsolutePath(); - LOGGER.info("new file {} {}", newFileName, entity.getOriginPattern()); + LOGGER.info("new file {} {}", newFileName, entity.getPattern()); Matcher matcher = entity.getPattern().matcher(newFileName); if (matcher.matches() || matcher.lookingAt()) { - LOGGER.info("matched file {} {}", newFileName, entity.getOriginPattern()); + LOGGER.info("matched file {} {}", newFileName, entity.getPattern()); String dataTime = getDataTimeFromFileName(newFileName, entity.getOriginPattern(), entity.getDateExpression()); if (!checkFileNameForTime(newFileName, entity)) { - LOGGER.error(AgentErrMsg.FILE_ERROR + "File Timeout {} {}", newFileName, dataTime); + LOGGER.error("File Timeout {} {}", newFileName, dataTime); return; } addToEvenMap(newFileName, dataTime); @@ -523,8 +384,8 @@ private void addToEvenMap(String fileName, String dataTime) { } else { cycleUnit = taskProfile.getCycleUnit(); } - InstanceProfile instanceProfile = taskProfile.createInstanceProfile(DEFAULT_FILE_INSTANCE, - fileName, cycleUnit, dataTime, fileUpdateTime); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(fileName, cycleUnit, dataTime, + fileUpdateTime); sameDataTimeEvents.put(fileName, instanceProfile); LOGGER.info("add to eventMap taskId {} dataTime {} fileName {}", taskProfile.getTaskId(), dataTime, fileName); } @@ -535,8 +396,7 @@ private boolean checkFileNameForTime(String newFileName, WatchEntity entity) { if (dateExpression.getLongestDatePattern().length() != 0) { String dataTime = getDataTimeFromFileName(newFileName, entity.getOriginPattern(), dateExpression); LOGGER.info("file {}, fileTime {}", newFileName, dataTime); - if (!NewDateUtils.isValidCreationTime(dataTime, entity.getCycleUnit(), - taskProfile.getTimeOffset())) { + if (!NewDateUtils.isValidCreationTime(dataTime, entity.getCycleUnit(), timeOffset)) { return false; } } @@ -566,8 +426,7 @@ private void resetWatchKey(WatchEntity entity, WatchKey key, Path contextPath) { * Register a new watch service on the path if the old watcher is invalid. */ if (!key.isValid()) { - LOGGER.warn(AgentErrMsg.WATCHER_INVALID + "Invalid Watcher {}", - contextPath.getFileName()); + LOGGER.warn("Invalid Watcher {}", contextPath.getFileName()); try { WatchService oldService = entity.getWatchService(); oldService.close(); diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/FileTimeComparator.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileTimeComparator.java similarity index 94% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/FileTimeComparator.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileTimeComparator.java index 949044d864a..aef1c9fd9ea 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/FileTimeComparator.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/FileTimeComparator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.utils.file; +package org.apache.inlong.agent.plugin.task.logcollection.local; import java.io.File; import java.util.Comparator; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/Files.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/Files.java similarity index 97% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/Files.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/Files.java index b4ddcfac530..12b085226d5 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/Files.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/Files.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.utils.file; +package org.apache.inlong.agent.plugin.task.logcollection.local; import org.apache.inlong.agent.utils.file.FileFinder; diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/WatchEntity.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/WatchEntity.java similarity index 94% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/WatchEntity.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/WatchEntity.java index af6d018a1d8..47d052e81b6 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/WatchEntity.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/logcollection/local/WatchEntity.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.task.file; +package org.apache.inlong.agent.plugin.task.logcollection.local; -import org.apache.inlong.agent.plugin.utils.file.DateUtils; -import org.apache.inlong.agent.plugin.utils.file.FilePathUtil; -import org.apache.inlong.agent.plugin.utils.file.NewDateUtils; -import org.apache.inlong.agent.plugin.utils.file.NonRegexPatternPosition; -import org.apache.inlong.agent.plugin.utils.file.PathDateExpression; +import org.apache.inlong.agent.plugin.utils.regex.DateUtils; +import org.apache.inlong.agent.plugin.utils.regex.NewDateUtils; +import org.apache.inlong.agent.plugin.utils.regex.NonRegexPatternPosition; +import org.apache.inlong.agent.plugin.utils.regex.PathDateExpression; +import org.apache.inlong.agent.plugin.utils.regex.PatternUtil; import org.apache.inlong.agent.utils.AgentUtils; import org.slf4j.Logger; @@ -69,11 +69,11 @@ public WatchEntity(WatchService watchService, String cycleUnit) { this.watchService = watchService; this.originPattern = originPattern; - ArrayList directoryLayers = FilePathUtil.cutDirectoryByWildcardAndDateExpression(originPattern); + ArrayList directoryLayers = PatternUtil.cutDirectoryByWildcardAndDateExpression(originPattern); this.basicStaticPath = directoryLayers.get(0); this.regexPattern = NewDateUtils.replaceDateExpressionWithRegex(originPattern); pattern = Pattern.compile(regexPattern, Pattern.CASE_INSENSITIVE | Pattern.DOTALL | Pattern.MULTILINE); - ArrayList directories = FilePathUtil.cutDirectoryByWildcard(originPattern); + ArrayList directories = PatternUtil.cutDirectoryByWildcard(originPattern); this.originPatternWithoutFileName = directories.get(0); this.patternWithoutFileName = Pattern .compile(NewDateUtils.replaceDateExpressionWithRegex(originPatternWithoutFileName), @@ -91,6 +91,7 @@ public WatchEntity(WatchService watchService, @Override public String toString() { return "WatchEntity [parentPathName=" + basicStaticPath + + ", pattern=" + pattern + ", readFilePattern=" + regexPattern + ", dateExpression=" + dateExpression + ", originPatternWithoutFileName=" + originPatternWithoutFileName + ", containRegexPattern=" diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/FormatDateLogFileTask.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/cos/COSConfigHandler.java similarity index 76% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/FormatDateLogFileTask.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/cos/COSConfigHandler.java index 697fad28618..259d3307763 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/FormatDateLogFileTask.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/cos/COSConfigHandler.java @@ -15,13 +15,12 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.task; +package org.apache.inlong.agent.plugin.utils.cos; -import org.apache.inlong.agent.plugin.task.file.LogFileTask; +import com.qcloud.cos.ClientConfig; -/** - * Directory trigger with format date. - */ -public class FormatDateLogFileTask extends LogFileTask { +// For some private, customized extension processing +public abstract class COSConfigHandler { + abstract public ClientConfig getClientConfig(String region); } diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/cos/COSUtils.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/cos/COSUtils.java new file mode 100644 index 00000000000..111e94212f3 --- /dev/null +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/cos/COSUtils.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.agent.plugin.utils.cos; + +import com.qcloud.cos.COSClient; +import com.qcloud.cos.ClientConfig; +import com.qcloud.cos.auth.BasicCOSCredentials; +import com.qcloud.cos.auth.COSCredentials; + +public class COSUtils { + + public static COSClient createCli(String secretId, String secretKey, String region) { + COSCredentials cred = new BasicCOSCredentials(secretId, secretKey); + COSConfigHandler configHandler = new DefaultCOSConfigHandler(); + ClientConfig clientConfig = configHandler.getClientConfig(region); + return new COSClient(cred, clientConfig); + } +} \ No newline at end of file diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/TaskType.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/cos/DefaultCOSConfigHandler.java similarity index 72% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/TaskType.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/cos/DefaultCOSConfigHandler.java index 65203d05225..a9dda478a29 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/task/file/TaskType.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/cos/DefaultCOSConfigHandler.java @@ -15,25 +15,14 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.task.file; +package org.apache.inlong.agent.plugin.utils.cos; -public enum TaskType { +import com.qcloud.cos.ClientConfig; +import com.qcloud.cos.region.Region; - READER(0), - TAILER(1), - UPLOADER(2), - STATE(3), - OTHER(4), - DB(5), - GAIAReader(6); +public class DefaultCOSConfigHandler extends COSConfigHandler { - private int type; - - TaskType(int type) { - this.type = type; - } - - public int getType() { - return type; + public ClientConfig getClientConfig(String region) { + return new ClientConfig(new Region(region)); } -} \ No newline at end of file +} diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/DateUtils.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/DateUtils.java similarity index 58% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/DateUtils.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/DateUtils.java index ae57acbca8a..2366dbae061 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/DateUtils.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/DateUtils.java @@ -15,16 +15,13 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.utils.file; +package org.apache.inlong.agent.plugin.utils.regex; import hirondelle.date4j.DateTime; import org.apache.commons.lang.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.text.ParseException; -import java.text.SimpleDateFormat; -import java.util.Date; import java.util.Objects; import java.util.TimeZone; import java.util.regex.Matcher; @@ -42,16 +39,6 @@ public class DateUtils { Pattern.CASE_INSENSITIVE | Pattern.DOTALL | Pattern.MULTILINE); private String dateFormat = "YYYYMMDDhhmmss"; - public DateUtils() { - - } - - public DateUtils(String timeFormat) { - if (timeFormat != null && !timeFormat.isEmpty()) { - dateFormat = timeFormat; - } - } - public static String getSubTimeFormat(String format, int length) { // format may be "YYYYMMDDhhmmss" | "YYYY_MM_DD_hh_mm_ss" int formatLen = format.length(); @@ -138,35 +125,6 @@ public static PathDateExpression extractLongestTimeRegexWithPrefixOrSuffix(Strin : new PathDateExpression(longestPattern, position)); } - public static String formatTime(long time) { - SimpleDateFormat df = new SimpleDateFormat("yyyyMMddHHmm"); - df.setTimeZone(TimeZone.getTimeZone("GMT+8:00")); - return df.format(new Date(time)); - } - - public static boolean compare(String time, int offset) - throws ParseException { - long value = 1000 * 60 * 60 * 24; - SimpleDateFormat df = new SimpleDateFormat("yyyyMMdd"); - long to = System.currentTimeMillis(); - long from = df.parse(time.substring(0, 8)).getTime(); - if ((to - from) / value > offset) { - return true; - } else { - return false; - } - } - - public static boolean compare(long time, int offset) { - long value = 1000 * 60 * 60 * 24; - long to = System.currentTimeMillis(); - if ((to - time) / value > offset) { - return true; - } else { - return false; - } - } - public void init(String timeFormat) { if (timeFormat != null && !timeFormat.isEmpty()) { dateFormat = timeFormat; @@ -221,38 +179,6 @@ private String normalizeDateStr(String src) { return sb.toString(); } - public String getFormatSpecifiedTime(String specifiedTime) { - if (specifiedTime == null || specifiedTime.length() == 0) { - return specifiedTime; - } - - int formatLen = dateFormat.length(); - - if (specifiedTime.length() == formatLen - && !specifiedTime.matches(DIGIT_STR)) { - return specifiedTime; - } - - StringBuilder retSb = new StringBuilder(); - int specifiedInx = 0; - for (int i = 0; i < formatLen; i++) { - char tmpChar = dateFormat.charAt(i); - - if (tmpChar != 'Y' && tmpChar != 'M' && tmpChar != 'D' - && tmpChar != 'h' && tmpChar != 'm') { - retSb.append(tmpChar); - } else { - retSb.append(specifiedTime.charAt(specifiedInx++)); - } - } - - logger.info( - "TimeRegex {} <> specifiedTime {} not match, format specifiedTime {}", - new Object[]{dateFormat, specifiedTime, retSb.toString()}); - - return retSb.toString(); - } - public String getDate(String src, String limit) { if (src == null || src.trim().isEmpty()) { return ""; @@ -333,123 +259,4 @@ public String getDate(String src, String limit) { } return dt.format(outputFormat); } - - public String getAttrPunit(String attrs) { - String punit = null; - if (attrs != null && attrs.contains("&p=")) { - for (String attr : attrs.split("&")) { - if (attr.startsWith("p=") && attr.split("=").length == 2) { - punit = attr.split("=")[1]; - break; - } - } - } - - return punit; - } - - public String getSpecifiedDate(String src, String limit, String punit) { - String ret = getDate(src, limit); - return formatCurrPeriod(ret, punit); - } - - public String normalizeTimeRegex(String src) { - return getSubTimeFormat(dateFormat, src.length()); - } - - public String getCurrentDir(String src, String timeOffset) { - Matcher m = pattern.matcher(src); - StringBuffer sb = new StringBuffer(); - while (m.find()) { - String oneMatch = m.group(0); - String currTimeStr = getDate(oneMatch, timeOffset); - m.appendReplacement(sb, currTimeStr); - } - m.appendTail(sb); - return sb.toString(); - } - - public String getCurrentDirByPunit(String src, String timeOffset, - String punit) { - Matcher m = pattern.matcher(src); - StringBuffer sb = new StringBuffer(); - while (m.find()) { - String oneMatch = m.group(0); - String currTimeStr = getSpecifiedDate(oneMatch, timeOffset, punit); - m.appendReplacement(sb, currTimeStr); - } - m.appendTail(sb); - - return sb.toString(); - } - - public String getSpecifiedDir(String src, String specifiedDate) { - Matcher m = pattern.matcher(src); - StringBuffer sb = new StringBuffer(); - - while (m.find()) { - String oneMatch = m.group(0); - StringBuilder tmpSb = new StringBuilder(); - int specifiedDateIdx = 0; - - for (int i = 0; i < oneMatch.length(); i++) { - char matchChar = oneMatch.charAt(i); - if (matchChar != 'Y' && matchChar != 'M' && matchChar != 'D' - && matchChar != 'h' && matchChar != 'm') { - tmpSb.append(matchChar); - } else { - char dateChar = specifiedDate.charAt(specifiedDateIdx); - while (String.valueOf(dateChar).matches("\\D")) { - dateChar = specifiedDate.charAt(++specifiedDateIdx); - } - tmpSb.append(dateChar); - specifiedDateIdx++; - } - } - m.appendReplacement(sb, tmpSb.toString()); - } - m.appendTail(sb); - return sb.toString(); - } - - // format current period starting less-than-hour task - // * for example: ten-minute task: - // * currPeriodDataTime is 201303271905 - // * formated value is 201303271900 - public String formatCurrPeriod(String src, String punit) { - if (src == null || punit == null || src.length() != 12) { - return src; - } - - String prefixMinuteStr = src.substring(0, src.length() - 2); - String minuteStr = src.substring(src.length() - 2, src.length()); - - if ("n".equals(punit)) { - if (minuteStr.compareTo("30") < 0) { - minuteStr = "00"; - } else { - minuteStr = "30"; - } - } else if ("q".equals(punit)) { - if (minuteStr.compareTo("15") < 0) { - minuteStr = "00"; - } else if (minuteStr.compareTo("30") < 0) { - minuteStr = "15"; - } else if (minuteStr.compareTo("45") < 0) { - minuteStr = "30"; - } else { - minuteStr = "45"; - } - } else if ("t".equals(punit)) { - minuteStr = minuteStr.charAt(0) + "0"; - } else if ("f".equals(punit)) { - if (minuteStr.substring(1).compareTo("5") < 0) { - minuteStr = minuteStr.charAt(0) + "0"; - } else { - minuteStr = minuteStr.charAt(0) + "5"; - } - } - - return prefixMinuteStr + minuteStr; - } } diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/MatchPoint.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/MatchPoint.java similarity index 96% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/MatchPoint.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/MatchPoint.java index b7222a191a0..ec1167d06e7 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/MatchPoint.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/MatchPoint.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.utils.file; +package org.apache.inlong.agent.plugin.utils.regex; public class MatchPoint { diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/NewDateUtils.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/NewDateUtils.java similarity index 92% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/NewDateUtils.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/NewDateUtils.java index 563a168885e..a80f088e0d6 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/NewDateUtils.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/NewDateUtils.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.utils.file; +package org.apache.inlong.agent.plugin.utils.regex; import org.apache.inlong.agent.constant.CycleUnitType; import org.apache.inlong.agent.utils.DateTransUtils; @@ -109,45 +109,6 @@ public static String getShouldStartTime(String dataTime, String cycleUnit, .format(new Date(getDateTime(calendar, cycleUnit, offset).getTimeInMillis())); } - private static Calendar getCurDate(String cycleUnit, String offset) { - if (cycleUnit == null || cycleUnit.length() == 0) { - return null; - } - - Calendar calendar = Calendar.getInstance(); - calendar.setTimeInMillis(System.currentTimeMillis()); - - return getDateTime(calendar, cycleUnit, offset); - } - - public static String getDateTime(String dataTime, String cycleUnit, String offset) { - String retTime = DateTransUtils.millSecConvertToTimeStr( - System.currentTimeMillis(), cycleUnit); - try { - long time = DateTransUtils.timeStrConvertToMillSec(dataTime, cycleUnit); - - Calendar calendar = Calendar.getInstance(); - calendar.setTimeInMillis(time); - Calendar retCalendar = getDateTime(calendar, cycleUnit, offset); - if (retCalendar == null) { - return dataTime; - } - - retTime = DateTransUtils.millSecConvertToTimeStr(retCalendar.getTime().getTime(), - cycleUnit); - } catch (Exception e) { - logger.error("getDateTime error: ", e); - } - return retTime; - } - - public static String getDateTime(long time, String cycleUnit, String offset) { - Calendar calendar = Calendar.getInstance(); - calendar.setTimeInMillis(time); - Calendar retCalendar = getDateTime(calendar, cycleUnit, offset); - return DateTransUtils.millSecConvertToTimeStr(retCalendar.getTime().getTime(), cycleUnit); - } - private static Calendar getDateTime(Calendar calendar, String cycleUnit, String offset) { int cycleNumber = (cycleUnit.length() <= 1 ? 1 diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/NonRegexPatternPosition.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/NonRegexPatternPosition.java similarity index 97% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/NonRegexPatternPosition.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/NonRegexPatternPosition.java index 0732d184e0d..a12140a3149 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/NonRegexPatternPosition.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/NonRegexPatternPosition.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.utils.file; +package org.apache.inlong.agent.plugin.utils.regex; /* * Describe the nearest character around the date time expression. For example, for date source name diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/PathDateExpression.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/PathDateExpression.java similarity index 96% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/PathDateExpression.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/PathDateExpression.java index c75e0398b8e..d73329424f5 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/PathDateExpression.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/PathDateExpression.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.utils.file; +package org.apache.inlong.agent.plugin.utils.regex; /* The date expression in the file path. */ public class PathDateExpression { diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/FilePathUtil.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/PatternUtil.java similarity index 91% rename from inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/FilePathUtil.java rename to inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/PatternUtil.java index 912b0356425..dc27addb072 100644 --- a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/file/FilePathUtil.java +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/PatternUtil.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.inlong.agent.plugin.utils.file; +package org.apache.inlong.agent.plugin.utils.regex; import org.apache.commons.lang.StringUtils; import java.io.File; import java.util.ArrayList; -public class FilePathUtil { +public class PatternUtil { private static final String YEAR = "YYYY"; private static final String MONTH = "MM"; @@ -158,9 +158,18 @@ public static ArrayList cutDirectoryByWildcardAndDateExpression(String d } public static boolean isSameDir(String fileName1, String fileName2) { - ArrayList ret1 = FilePathUtil.cutDirectoryByWildcard(fileName1); - ArrayList ret2 = FilePathUtil.cutDirectoryByWildcard(fileName2); + ArrayList ret1 = PatternUtil.cutDirectoryByWildcard(fileName1); + ArrayList ret2 = PatternUtil.cutDirectoryByWildcard(fileName2); return ret1.get(0).equals(ret2.get(0)); } + public static String getBeforeFirstWildcard(String input) { + String sign = "\\^$*+?{(|[."; + int firstWildcardIndex = StringUtils.indexOfAny(input, sign); + if (firstWildcardIndex != -1) { + return input.substring(0, firstWildcardIndex); + } else { + return ""; + } + } } diff --git a/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/Scanner.java b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/Scanner.java new file mode 100644 index 00000000000..b3153e81fef --- /dev/null +++ b/inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/utils/regex/Scanner.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.agent.plugin.utils.regex; + +import org.apache.inlong.agent.utils.DateTransUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Calendar; +import java.util.List; + +public class Scanner { + + private static final Logger LOGGER = LoggerFactory.getLogger(Scanner.class); + public static final String SCAN_CYCLE_RANCE = "-2"; + + public static class TimeRange { + + public Long startTime; + public Long endTime; + + public TimeRange(Long startTime, Long endTime) { + this.startTime = startTime; + this.endTime = endTime; + } + } + + public static class FinalPatternInfo { + + public String finalPattern; + public Long dataTime; + + public FinalPatternInfo(String finalPattern, Long dataTime) { + this.finalPattern = finalPattern; + this.dataTime = dataTime; + } + } + + public static List getFinalPatternInfos(String originPattern, String cycleUnit, String timeOffset, + long startTime, long endTime, boolean isRetry) { + TimeRange range = Scanner.getTimeRange(startTime, endTime, cycleUnit, timeOffset, isRetry); + String strStartTime = DateTransUtils.millSecConvertToTimeStr(range.startTime, cycleUnit); + String strEndTime = DateTransUtils.millSecConvertToTimeStr(range.endTime, cycleUnit); + LOGGER.info("{} scan time is between {} and {}", originPattern, strStartTime, strEndTime); + List dateRegion = NewDateUtils.getDateRegion(range.startTime, range.endTime, cycleUnit); + List finalPatternList = new ArrayList<>(); + for (Long time : dateRegion) { + Calendar calendar = Calendar.getInstance(); + calendar.setTimeInMillis(time); + FinalPatternInfo finalPatternInfo = new FinalPatternInfo( + NewDateUtils.replaceDateExpression(calendar, originPattern), time); + finalPatternList.add(finalPatternInfo); + } + return finalPatternList; + } + + public static List getDataTimeList(long startTime, long endTime, String cycleUnit, String timeOffset, + boolean isRetry) { + TimeRange range = getTimeRange(startTime, endTime, cycleUnit, timeOffset, isRetry); + List dataTimeList = new ArrayList<>(); + List dateRegion = NewDateUtils.getDateRegion(range.startTime, range.endTime, cycleUnit); + for (Long time : dateRegion) { + String dataTime = DateTransUtils.millSecConvertToTimeStr(time, cycleUnit); + dataTimeList.add(dataTime); + } + return dataTimeList; + } + + public static TimeRange getTimeRange(long startTime, long endTime, String cycleUnit, String timeOffset, + boolean isRetry) { + if (!isRetry) { + long currentTime = System.currentTimeMillis(); + // only scan two cycle, like two hours or two days + long offset = DateTransUtils.calcOffset(SCAN_CYCLE_RANCE + cycleUnit); + startTime = currentTime + offset + DateTransUtils.calcOffset(timeOffset); + endTime = currentTime + DateTransUtils.calcOffset(timeOffset); + } + return new TimeRange(startTime, endTime); + } +} diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/AgentBaseTestsHelper.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/AgentBaseTestsHelper.java index 7730bb29c84..214a29b24e7 100755 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/AgentBaseTestsHelper.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/AgentBaseTestsHelper.java @@ -21,8 +21,10 @@ import org.apache.inlong.agent.conf.TaskProfile; import org.apache.inlong.agent.constant.AgentConstants; import org.apache.inlong.agent.constant.FetcherConstants; +import org.apache.inlong.agent.pojo.COSTask.COSTaskConfig; import org.apache.inlong.agent.pojo.FileTask.FileTaskConfig; import org.apache.inlong.common.enums.TaskStateEnum; +import org.apache.inlong.common.enums.TaskTypeEnum; import org.apache.inlong.common.pojo.agent.DataConfig; import com.google.gson.Gson; @@ -82,24 +84,24 @@ public void teardownAgentHome() { } } - public TaskProfile getTaskProfile(int taskId, String pattern, String dataContentStyle, boolean retry, - Long startTime, Long endTime, + public TaskProfile getFileTaskProfile(int taskId, String pattern, String dataContentStyle, boolean retry, + String startTime, String endTime, TaskStateEnum state, String cycleUnit, String timeZone, List filterStreams) { - DataConfig dataConfig = getDataConfig(taskId, pattern, dataContentStyle, retry, startTime, endTime, + DataConfig dataConfig = getFileDataConfig(taskId, pattern, dataContentStyle, retry, startTime, endTime, state, cycleUnit, timeZone, filterStreams); TaskProfile profile = TaskProfile.convertToTaskProfile(dataConfig); return profile; } - private DataConfig getDataConfig(int taskId, String pattern, String dataContentStyle, boolean retry, Long startTime, - Long endTime, - TaskStateEnum state, String cycleUnit, String timeZone, List filterStreams) { + private DataConfig getFileDataConfig(int taskId, String pattern, String dataContentStyle, boolean retry, + String startTime, String endTime, TaskStateEnum state, String cycleUnit, String timeZone, + List filterStreams) { DataConfig dataConfig = new DataConfig(); dataConfig.setInlongGroupId("testGroupId"); dataConfig.setInlongStreamId("testStreamId"); dataConfig.setDataReportType(1); - dataConfig.setTaskType(3); + dataConfig.setTaskType(TaskTypeEnum.FILE.getType()); dataConfig.setTaskId(taskId); dataConfig.setTimeZone(timeZone); dataConfig.setState(state.ordinal()); @@ -110,8 +112,8 @@ private DataConfig getDataConfig(int taskId, String pattern, String dataContentS fileTaskConfig.setMaxFileCount(100); fileTaskConfig.setCycleUnit(cycleUnit); fileTaskConfig.setRetry(retry); - fileTaskConfig.setStartTime(startTime); - fileTaskConfig.setEndTime(endTime); + fileTaskConfig.setDataTimeFrom(startTime); + fileTaskConfig.setDataTimeTo(endTime); // mix: login|87601|968|67826|23579 or login|a=b&c=d&x=y&asdf fileTaskConfig.setDataContentStyle(dataContentStyle); fileTaskConfig.setDataSeparator("|"); @@ -119,4 +121,47 @@ private DataConfig getDataConfig(int taskId, String pattern, String dataContentS dataConfig.setExtParams(GSON.toJson(fileTaskConfig)); return dataConfig; } + + public TaskProfile getCOSTaskProfile(int taskId, String pattern, String contentStyle, boolean retry, + String startTime, String endTime, + TaskStateEnum state, String cycleUnit, String timeZone, List filterStreams) { + DataConfig dataConfig = getCOSDataConfig(taskId, pattern, contentStyle, retry, startTime, endTime, + state, cycleUnit, timeZone, + filterStreams); + TaskProfile profile = TaskProfile.convertToTaskProfile(dataConfig); + return profile; + } + + private DataConfig getCOSDataConfig(int taskId, String pattern, String contentStyle, boolean retry, + String startTime, String endTime, TaskStateEnum state, String cycleUnit, String timeZone, + List filterStreams) { + DataConfig dataConfig = new DataConfig(); + dataConfig.setInlongGroupId("testGroupId"); + dataConfig.setInlongStreamId("testStreamId"); + dataConfig.setDataReportType(1); + dataConfig.setTaskType(TaskTypeEnum.COS.getType()); + dataConfig.setTaskId(taskId); + dataConfig.setTimeZone(timeZone); + dataConfig.setState(state.ordinal()); + COSTaskConfig cosTaskConfig = new COSTaskConfig(); + cosTaskConfig.setBucketName("testBucket"); + cosTaskConfig.setCredentialsId("testSecretId"); + cosTaskConfig.setCredentialsKey("testSecretKey"); + cosTaskConfig.setRegion("testRegion"); + cosTaskConfig.setPattern(pattern); + cosTaskConfig.setTimeOffset("0d"); + // GMT-8:00 same with Asia/Shanghai + cosTaskConfig.setMaxFileCount(100); + cosTaskConfig.setCycleUnit(cycleUnit); + cosTaskConfig.setRetry(retry); + cosTaskConfig.setDataTimeFrom(startTime); + cosTaskConfig.setDataTimeTo(endTime); + // mix: login|87601|968|67826|23579 or login|a=b&c=d&x=y&asdf + cosTaskConfig.setContentStyle(contentStyle); + cosTaskConfig.setDataSeparator("|"); + cosTaskConfig.setFilterStreams(filterStreams); + dataConfig.setExtParams(GSON.toJson(cosTaskConfig)); + return dataConfig; + } + } diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/instance/MockInstance.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/instance/MockInstance.java index d3dc67df5ca..278a9298f96 100644 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/instance/MockInstance.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/instance/MockInstance.java @@ -55,6 +55,11 @@ public void destroy() { destroyTime = index.getAndAdd(1); } + @Override + public void notifyDestroy() { + + } + @Override public InstanceProfile getProfile() { return profile; diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/instance/TestInstanceManager.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/instance/TestInstanceManager.java index f2c5f25ee3d..4340bd6176c 100755 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/instance/TestInstanceManager.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/instance/TestInstanceManager.java @@ -38,6 +38,12 @@ import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +53,9 @@ import static org.awaitility.Awaitility.await; +@RunWith(PowerMockRunner.class) +@PrepareForTest(InstanceProfile.class) +@PowerMockIgnore({"javax.management.*"}) public class TestInstanceManager { private static final Logger LOGGER = LoggerFactory.getLogger(TestInstanceManager.class); @@ -56,11 +65,21 @@ public class TestInstanceManager { @BeforeClass public static void setup() { + PowerMockito.mockStatic(InstanceProfile.class); + Mockito.when(InstanceProfile.getInstanceClassByTaskType(Mockito.any())) + .thenAnswer(mock -> MockInstance.class.getCanonicalName()); + Mockito.when(InstanceProfile.parseJsonStr(Mockito.anyString())).thenAnswer(mock -> { + String jsonStr = mock.getArgument(0); + InstanceProfile conf = new InstanceProfile(); + conf.loadJsonStrResource(jsonStr); + return conf; + }); helper = new AgentBaseTestsHelper(TestInstanceManager.class.getName()).setupAgentHome(); String pattern = helper.getTestRootDir() + "/YYYYMMDDhh_[0-9]+.txt"; Store basicInstanceStore = TaskManager.initStore(AgentConstants.AGENT_STORE_PATH_INSTANCE); - taskProfile = helper.getTaskProfile(1, pattern, "csv", false, 0L, 0L, TaskStateEnum.RUNNING, CycleUnitType.HOUR, - "GMT+6:00", null); + taskProfile = + helper.getFileTaskProfile(1, pattern, "csv", false, "", "", TaskStateEnum.RUNNING, CycleUnitType.HOUR, + "GMT+6:00", null); Store taskBasicStore = TaskManager.initStore(AgentConstants.AGENT_STORE_PATH_TASK); TaskStore taskStore = new TaskStore(taskBasicStore); taskStore.storeTask(taskProfile); @@ -78,9 +97,8 @@ public static void teardown() { public void testInstanceManager() { InstanceStore instanceStore = manager.getInstanceStore(); for (int i = 1; i <= 10; i++) { - InstanceProfile profile = taskProfile.createInstanceProfile(MockInstance.class.getCanonicalName(), - String.valueOf(i), taskProfile.getCycleUnit(), "2023092710", - AgentUtils.getCurrentTime()); + InstanceProfile profile = taskProfile.createInstanceProfile(String.valueOf(i), taskProfile.getCycleUnit(), + "2023092710", AgentUtils.getCurrentTime()); instanceStore.storeInstance(profile); } manager.start(); @@ -90,9 +108,8 @@ public void testInstanceManager() { Assert.assertTrue(manager.getInstanceProfile(instanceId).getState() == InstanceStateEnum.DEFAULT); } long timeBefore = AgentUtils.getCurrentTime(); - InstanceProfile profile = taskProfile.createInstanceProfile(MockInstance.class.getCanonicalName(), - helper.getTestRootDir() + "/2023092710_1.txt", taskProfile.getCycleUnit(), "2023092710", - AgentUtils.getCurrentTime()); + InstanceProfile profile = taskProfile.createInstanceProfile(helper.getTestRootDir() + "/2023092710_1.txt", + taskProfile.getCycleUnit(), "2023092710", AgentUtils.getCurrentTime()); String sinkDataTime = String.valueOf(profile.getSinkDataTime()); try { String add2TimeZone = String.valueOf( @@ -121,9 +138,8 @@ public void testInstanceManager() { Assert.assertTrue(manager.shouldAddAgain(profile.getInstanceId(), AgentUtils.getCurrentTime())); // test continue - profile = taskProfile.createInstanceProfile(MockInstance.class.getCanonicalName(), - helper.getTestRootDir() + "/2023092710_1.txt", taskProfile.getCycleUnit(), "2023092710", - AgentUtils.getCurrentTime()); + profile = taskProfile.createInstanceProfile(helper.getTestRootDir() + "/2023092710_1.txt", + taskProfile.getCycleUnit(), "2023092710", AgentUtils.getCurrentTime()); action = new InstanceAction(); action.setActionType(ActionType.ADD); action.setProfile(profile); diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/KafkaSinkTest.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/KafkaSinkTest.java index 5524c5e96e3..56f46d74a06 100644 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/KafkaSinkTest.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/KafkaSinkTest.java @@ -47,10 +47,11 @@ public static void setUp() throws Exception { String fileName = LOADER.getResource("test/20230928_1.txt").getPath(); helper = new AgentBaseTestsHelper(TestSenderManager.class.getName()).setupAgentHome(); String pattern = helper.getTestRootDir() + "/YYYYMMDD.log_[0-9]+"; - TaskProfile taskProfile = helper.getTaskProfile(1, pattern, "csv", false, 0L, 0L, TaskStateEnum.RUNNING, "D", - "GMT+8:00", null); - profile = taskProfile.createInstanceProfile("", fileName, - taskProfile.getCycleUnit(), "20230927", AgentUtils.getCurrentTime()); + TaskProfile taskProfile = + helper.getFileTaskProfile(1, pattern, "csv", false, "", "", TaskStateEnum.RUNNING, "D", + "GMT+8:00", null); + profile = taskProfile.createInstanceProfile(fileName, taskProfile.getCycleUnit(), "20230927", + AgentUtils.getCurrentTime()); kafkaSink = new MockSink(); kafkaSink.init(profile); } diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/PulsarSinkTest.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/PulsarSinkTest.java index 93702fad169..c8cf3658506 100644 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/PulsarSinkTest.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/PulsarSinkTest.java @@ -47,10 +47,11 @@ public static void setUp() throws Exception { String fileName = LOADER.getResource("test/20230928_1.txt").getPath(); helper = new AgentBaseTestsHelper(TestSenderManager.class.getName()).setupAgentHome(); String pattern = helper.getTestRootDir() + "/YYYYMMDD.log_[0-9]+"; - TaskProfile taskProfile = helper.getTaskProfile(1, pattern, "csv", false, 0L, 0L, TaskStateEnum.RUNNING, "D", - "GMT+8:00", null); - profile = taskProfile.createInstanceProfile("", fileName, - taskProfile.getCycleUnit(), "20230927", AgentUtils.getCurrentTime()); + TaskProfile taskProfile = + helper.getFileTaskProfile(1, pattern, "csv", false, "", "", TaskStateEnum.RUNNING, "D", + "GMT+8:00", null); + profile = taskProfile.createInstanceProfile(fileName, taskProfile.getCycleUnit(), "20230927", + AgentUtils.getCurrentTime()); pulsarSink = new MockSink(); pulsarSink.init(profile); } diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/filecollect/TestSenderManager.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/filecollect/TestSenderManager.java index afeb3565e21..9655e757ef9 100644 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/filecollect/TestSenderManager.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sinks/filecollect/TestSenderManager.java @@ -24,7 +24,7 @@ import org.apache.inlong.agent.message.file.OffsetAckInfo; import org.apache.inlong.agent.message.file.SenderMessage; import org.apache.inlong.agent.plugin.AgentBaseTestsHelper; -import org.apache.inlong.agent.plugin.utils.file.FileDataUtils; +import org.apache.inlong.agent.plugin.task.logcollection.local.FileDataUtils; import org.apache.inlong.agent.utils.AgentUtils; import org.apache.inlong.common.enums.TaskStateEnum; import org.apache.inlong.sdk.dataproxy.common.SendMessageCallback; @@ -70,10 +70,11 @@ public static void setup() { String fileName = LOADER.getResource("test/20230928_1.txt").getPath(); helper = new AgentBaseTestsHelper(TestSenderManager.class.getName()).setupAgentHome(); String pattern = helper.getTestRootDir() + "/YYYYMMDD.log_[0-9]+"; - TaskProfile taskProfile = helper.getTaskProfile(1, pattern, "csv", false, 0L, 0L, TaskStateEnum.RUNNING, "D", - "GMT+8:00", null); - profile = taskProfile.createInstanceProfile("", fileName, - taskProfile.getCycleUnit(), "20230927", AgentUtils.getCurrentTime()); + TaskProfile taskProfile = + helper.getFileTaskProfile(1, pattern, "csv", false, "", "", TaskStateEnum.RUNNING, "D", + "GMT+8:00", null); + profile = taskProfile.createInstanceProfile(fileName, taskProfile.getCycleUnit(), "20230927", + AgentUtils.getCurrentTime()); } @AfterClass diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestLogFileSource.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestLogFileSource.java index df700394592..2bbe41859af 100644 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestLogFileSource.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestLogFileSource.java @@ -27,7 +27,7 @@ import org.apache.inlong.agent.core.task.TaskManager; import org.apache.inlong.agent.plugin.AgentBaseTestsHelper; import org.apache.inlong.agent.plugin.Message; -import org.apache.inlong.agent.plugin.utils.file.FileDataUtils; +import org.apache.inlong.agent.plugin.task.logcollection.local.FileDataUtils; import org.apache.inlong.agent.store.Store; import org.apache.inlong.agent.utils.AgentUtils; import org.apache.inlong.common.enums.TaskStateEnum; @@ -42,6 +42,7 @@ import org.slf4j.LoggerFactory; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.concurrent.TimeUnit; import static org.apache.inlong.agent.constant.FetcherConstants.AGENT_GLOBAL_READER_QUEUE_PERMIT; @@ -74,15 +75,20 @@ public static void setup() { OffsetManager.init(taskBasicStore, instanceBasicStore, offsetBasicStore); } - private LogFileSource getSource(int taskId, long offset) { + private LogFileSource getSource(int taskId, long lineOffset, long byteOffset, String dataContentStyle, + boolean isOffSetNew) { try { - String pattern = helper.getTestRootDir() + "/YYYYMMDD.log_[0-9]+"; - TaskProfile taskProfile = helper.getTaskProfile(taskId, pattern, "csv", false, 0L, 0L, + String pattern; + String fileName; + boolean retry; + fileName = LOADER.getResource("test/20230928_1.txt").getPath(); + pattern = helper.getTestRootDir() + "/YYYYMMDD.log_[0-9]+"; + retry = false; + TaskProfile taskProfile = helper.getFileTaskProfile(taskId, pattern, dataContentStyle, retry, "", "", TaskStateEnum.RUNNING, "D", - "GMT+8:00", null); - String fileName = LOADER.getResource("test/20230928_1.txt").getPath(); - InstanceProfile instanceProfile = taskProfile.createInstanceProfile("", - fileName, taskProfile.getCycleUnit(), "20230928", AgentUtils.getCurrentTime()); + "GMT+8:00", Arrays.asList("ok")); + InstanceProfile instanceProfile = taskProfile.createInstanceProfile(fileName, taskProfile.getCycleUnit(), + "20230928", AgentUtils.getCurrentTime()); instanceProfile.set(TaskConstants.INODE_INFO, FileDataUtils.getInodeInfo(instanceProfile.getInstanceId())); LogFileSource source = new LogFileSource(); Whitebox.setInternalState(source, "BATCH_READ_LINE_COUNT", 1); @@ -90,18 +96,22 @@ private LogFileSource getSource(int taskId, long offset) { Whitebox.setInternalState(source, "CORE_THREAD_PRINT_INTERVAL_MS", 0); Whitebox.setInternalState(source, "SIZE_OF_BUFFER_TO_READ_FILE", 2); Whitebox.setInternalState(source, "EMPTY_CHECK_COUNT_AT_LEAST", 3); - Whitebox.setInternalState(source, "READ_WAIT_TIMEOUT_MS", 10); - if (offset > 0) { + Whitebox.setInternalState(source, "WAIT_TIMEOUT_MS", 10); + if (lineOffset > 0) { + String finalOffset = Long.toString(lineOffset); + if (isOffSetNew) { + finalOffset += LogFileSource.OFFSET_SEP + byteOffset; + } OffsetProfile offsetProfile = new OffsetProfile(instanceProfile.getTaskId(), instanceProfile.getInstanceId(), - Long.toString(offset), instanceProfile.get(INODE_INFO)); + finalOffset, instanceProfile.get(INODE_INFO)); OffsetManager.getInstance().setOffset(offsetProfile); } source.init(instanceProfile); source.start(); return source; } catch (Exception e) { - LOGGER.error("source init error {}", e); + LOGGER.error("source init error", e); Assert.assertTrue("source init error", false); } return null; @@ -124,7 +134,7 @@ private void testFullRead() { for (int i = 0; i < check.length; i++) { srcLen += check[i].getBytes(StandardCharsets.UTF_8).length; } - LogFileSource source = getSource(1, 0); + LogFileSource source = getSource(1, 0, 0, "csv", false); Message msg = source.read(); int readLen = 0; int cnt = 0; @@ -149,7 +159,7 @@ private void testFullRead() { } private void testCleanQueue() { - LogFileSource source = getSource(2, 0); + LogFileSource source = getSource(2, 0, 0, "csv", false); for (int i = 0; i < 2; i++) { source.read(); } @@ -160,16 +170,16 @@ private void testCleanQueue() { } private void testReadWithOffset() { - LogFileSource source = getSource(3, 1); + LogFileSource source = getSource(3, 1, 25, "csv", false); for (int i = 0; i < 2; i++) { Message msg = source.read(); - Assert.assertTrue(msg != null); + Assert.assertEquals(new String(msg.getBody()), check[i + 1]); } Message msg = source.read(); Assert.assertTrue(msg == null); source.destroy(); - source = getSource(4, 3); + source = getSource(4, 3, 69, "csv", false); msg = source.read(); Assert.assertTrue(msg == null); source.destroy(); diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestRedisSource.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestRedisSource.java index 2680c01e063..8fb551a0838 100644 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestRedisSource.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestRedisSource.java @@ -121,10 +121,10 @@ private void initProfile() { final String command = "zscore"; final String subOperation = "set,del"; - TaskProfile taskProfile = helper.getTaskProfile(1, "", "csv", false, 0L, 0L, TaskStateEnum.RUNNING, "D", + TaskProfile taskProfile = helper.getFileTaskProfile(1, "", "csv", false, "", "", TaskStateEnum.RUNNING, "D", "GMT+8:00", null); - profile = taskProfile.createInstanceProfile("", - "", taskProfile.getCycleUnit(), "20240725", AgentUtils.getCurrentTime()); + profile = taskProfile.createInstanceProfile("", taskProfile.getCycleUnit(), "20240725", + AgentUtils.getCurrentTime()); profile.set(CommonConstants.PROXY_INLONG_GROUP_ID, groupId); profile.set(CommonConstants.PROXY_INLONG_STREAM_ID, streamId); profile.set(TaskConstants.TASK_REDIS_AUTHUSER, username); diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestSQLServerSource.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestSQLServerSource.java index 2a90bdc37af..960f2a51442 100644 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestSQLServerSource.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/sources/TestSQLServerSource.java @@ -136,10 +136,10 @@ private SQLServerSource getSource() { final String tableName = "test_source"; final String serverName = "server-01"; - TaskProfile taskProfile = helper.getTaskProfile(1, "", "csv", false, 0L, 0L, TaskStateEnum.RUNNING, "D", + TaskProfile taskProfile = helper.getFileTaskProfile(1, "", "csv", false, "", "", TaskStateEnum.RUNNING, "D", "GMT+8:00", null); - instanceProfile = taskProfile.createInstanceProfile("", - "", taskProfile.getCycleUnit(), "20240725", AgentUtils.getCurrentTime()); + instanceProfile = taskProfile.createInstanceProfile("", taskProfile.getCycleUnit(), "20240725", + AgentUtils.getCurrentTime()); instanceProfile.set(CommonConstants.PROXY_INLONG_GROUP_ID, groupId); instanceProfile.set(CommonConstants.PROXY_INLONG_STREAM_ID, streamId); instanceProfile.set(TaskConstants.TASK_SQLSERVER_USER, username); diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestCOSTask.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestCOSTask.java new file mode 100644 index 00000000000..4d3bc0508db --- /dev/null +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestCOSTask.java @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.agent.plugin.task; + +import org.apache.inlong.agent.common.AgentThreadFactory; +import org.apache.inlong.agent.conf.TaskProfile; +import org.apache.inlong.agent.constant.CycleUnitType; +import org.apache.inlong.agent.core.task.TaskManager; +import org.apache.inlong.agent.plugin.AgentBaseTestsHelper; +import org.apache.inlong.agent.plugin.task.logcollection.cos.COSTask; +import org.apache.inlong.agent.plugin.utils.cos.COSUtils; +import org.apache.inlong.common.enums.TaskStateEnum; + +import com.qcloud.cos.COSClient; +import com.qcloud.cos.model.COSObjectSummary; +import com.qcloud.cos.model.ListObjectsRequest; +import com.qcloud.cos.model.ObjectListing; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static org.awaitility.Awaitility.await; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({COSUtils.class, COSTask.class, COSClient.class, ObjectListing.class}) +@PowerMockIgnore({"javax.management.*"}) +public class TestCOSTask { + + private static final Logger LOGGER = LoggerFactory.getLogger(TestCOSTask.class); + private static final ClassLoader LOADER = TestCOSTask.class.getClassLoader(); + private static AgentBaseTestsHelper helper; + private static TaskManager manager; + private static final ThreadPoolExecutor EXECUTOR_SERVICE = new ThreadPoolExecutor( + 0, Integer.MAX_VALUE, + 1L, TimeUnit.SECONDS, + new SynchronousQueue<>(), + new AgentThreadFactory("TestCOSTask")); + private static COSClient cosClient; + + @BeforeClass + public static void setup() throws Exception { + helper = new AgentBaseTestsHelper(TestCOSTask.class.getName()).setupAgentHome(); + manager = new TaskManager(); + cosClient = Mockito.mock(COSClient.class); + PowerMockito.mockStatic(COSUtils.class); + Mockito.when(COSUtils.createCli(Mockito.anyString(), Mockito.anyString(), Mockito.anyString())) + .thenReturn(cosClient); + } + + @AfterClass + public static void teardown() throws Exception { + helper.teardownAgentHome(); + } + + private void mockDay(COSClient cosClient) { + ObjectListing objectListing1_1 = Mockito.mock(ObjectListing.class); + when(objectListing1_1.getCommonPrefixes()).thenReturn( + Arrays.asList("some/20230928_0/", "some/20230928_1/", "some/20230928_aaa/")); + when(objectListing1_1.getObjectSummaries()).thenReturn(getSummaries(Arrays.asList("some/20230928_test_0.txt"))); + + ObjectListing objectListing1_2 = Mockito.mock(ObjectListing.class); + when(objectListing1_2.getCommonPrefixes()).thenReturn( + Arrays.asList("some/20230929_aaa/", "some/20230929_1/", "some/20230929_2/")); + when(objectListing1_2.getObjectSummaries()).thenReturn( + getSummaries(Arrays.asList("some/20230929_0_test_0.txt"))); + + ObjectListing objectListing2_1 = Mockito.mock(ObjectListing.class); + when(objectListing2_1.getCommonPrefixes()).thenReturn( + Arrays.asList("some/20230928_0/where/", "some/20230928_0/test_1/")); + when(objectListing2_1.getObjectSummaries()).thenReturn(getSummaries( + Arrays.asList("some/20230928_0/test_0.txt", "some/20230928_0/test_1.txt", + "some/20230928_0/test_o.txt"))); + + ObjectListing objectListing2_2 = Mockito.mock(ObjectListing.class); + when(objectListing2_2.getCommonPrefixes()).thenReturn( + Arrays.asList("some/20230929_1/where/", "some/20230929_1/test_1/")); + when(objectListing2_2.getObjectSummaries()).thenReturn(getSummaries( + Arrays.asList("some/20230929_1/test_0.txt", "some/20230929_1/test_1.txt", + "some/20230929_1/test_o.txt"))); + + when(cosClient.listObjects(Mockito.any(ListObjectsRequest.class))).thenAnswer(mock -> { + ListObjectsRequest req = mock.getArgument(0); + if (req.getPrefix().equals("some/20230928_")) { + return objectListing1_1; + } else if (req.getPrefix().equals("some/20230929_")) { + return objectListing1_2; + } else if (req.getPrefix().equals("some/20230928_0/")) { + return objectListing2_1; + } else if (req.getPrefix().equals("some/20230929_1/")) { + return objectListing2_2; + } else { + return new ObjectListing(); + } + }); + } + + private void mockHour(COSClient cosClient) { + ObjectListing objectListing1_1 = Mockito.mock(ObjectListing.class); + when(objectListing1_1.getCommonPrefixes()).thenReturn( + Arrays.asList("some/2023092800_0/", "some/2023092800_1/", "some/2023092800_aaa/")); + when(objectListing1_1.getObjectSummaries()).thenReturn( + getSummaries(Arrays.asList("some/2023092800_test_0.txt"))); + + ObjectListing objectListing1_2 = Mockito.mock(ObjectListing.class); + when(objectListing1_2.getCommonPrefixes()).thenReturn( + Arrays.asList("some/2023092901_aaa/", "some/2023092901_1/", "some/2023092901_2/")); + when(objectListing1_2.getObjectSummaries()).thenReturn( + getSummaries(Arrays.asList("some/2023092901_0_test_0.txt"))); + + ObjectListing objectListing2_1 = Mockito.mock(ObjectListing.class); + when(objectListing2_1.getCommonPrefixes()).thenReturn( + Arrays.asList("some/2023092800_0/where/", "some/2023092800_0/test_1/")); + when(objectListing2_1.getObjectSummaries()).thenReturn(getSummaries( + Arrays.asList("some/2023092800_0/test_0.txt", "some/2023092800_0/test_1.txt", + "some/2023092800_0/test_o.txt"))); + + ObjectListing objectListing2_2 = Mockito.mock(ObjectListing.class); + when(objectListing2_2.getCommonPrefixes()).thenReturn( + Arrays.asList("some/2023092901_1/where/", "some/2023092901_1/test_1/")); + when(objectListing2_2.getObjectSummaries()).thenReturn(getSummaries( + Arrays.asList("some/2023092901_1/test_0.txt", "some/2023092901_1/test_1.txt", + "some/2023092901_1/test_o.txt"))); + + when(cosClient.listObjects(Mockito.any(ListObjectsRequest.class))).thenAnswer(mock -> { + ListObjectsRequest req = mock.getArgument(0); + if (req.getPrefix().equals("some/2023092800_")) { + return objectListing1_1; + } else if (req.getPrefix().equals("some/2023092901_")) { + return objectListing1_2; + } else if (req.getPrefix().equals("some/2023092800_0/")) { + return objectListing2_1; + } else if (req.getPrefix().equals("some/2023092901_1/")) { + return objectListing2_2; + } else { + return new ObjectListing(); + } + }); + } + + private List getSummaries(List keys) { + List summaries = new ArrayList<>(); + for (int i = 0; i < keys.size(); i++) { + COSObjectSummary summary = new COSObjectSummary(); + summary.setKey(keys.get(i)); + summary.setSize(100); + summary.setStorageClass("what"); + summaries.add(summary); + } + return summaries; + } + + @Test + public void testScan() { + mockDay(cosClient); + doTest(1, "some/YYYYMMDD_[0-9]+/test_[0-9]+.txt", CycleUnitType.DAY, + Arrays.asList("some/20230928_0/test_0.txt", "some/20230928_0/test_1.txt", "some/20230929_1/test_0.txt", + "some/20230929_1/test_1.txt"), + Arrays.asList("20230928", "20230928", "20230929", "20230929"), + "20230928", + "20230930"); + mockHour(cosClient); + doTest(2, "some/YYYYMMDDhh_[0-9]+/test_[0-9]+.txt", CycleUnitType.HOUR, + Arrays.asList("some/2023092800_0/test_0.txt", "some/2023092800_0/test_1.txt", + "some/2023092901_1/test_0.txt", + "some/2023092901_1/test_1.txt"), + Arrays.asList("2023092800", "2023092800", "2023092901", "2023092901"), "2023092800", + "2023093023"); + } + + private void doTest(int taskId, String pattern, String cycle, List srcKeys, List srcDataTimes, + String startTime, String endTime) { + TaskProfile taskProfile = helper.getCOSTaskProfile(taskId, pattern, "csv", true, startTime, endTime, + TaskStateEnum.RUNNING, + cycle, "GMT+8:00", null); + COSTask fileTask = null; + final List fileName = new ArrayList(); + final List dataTime = new ArrayList(); + try { + fileTask = PowerMockito.spy(new COSTask()); + PowerMockito.doAnswer(invocation -> { + fileName.add(invocation.getArgument(0)); + dataTime.add(invocation.getArgument(1)); + return null; + }).when(fileTask, "addToEvenMap", Mockito.anyString(), Mockito.anyString()); + Assert.assertTrue(fileTask.isProfileValid(taskProfile)); + manager.getTaskStore().storeTask(taskProfile); + fileTask.init(manager, taskProfile, manager.getInstanceBasicStore()); + EXECUTOR_SERVICE.submit(fileTask); + } catch (Exception e) { + LOGGER.error("source init error", e); + Assert.assertTrue("source init error", false); + } + await().atMost(10, TimeUnit.SECONDS) + .until(() -> fileName.size() == srcDataTimes.size() && dataTime.size() == srcDataTimes.size()); + for (int i = 0; i < fileName.size(); i++) { + Assert.assertEquals(0, fileName.get(i).compareTo(srcKeys.get(i))); + Assert.assertEquals(0, dataTime.get(i).compareTo(srcDataTimes.get(i))); + } + fileTask.destroy(); + } +} \ No newline at end of file diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestLogFileTask.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestLogFileTask.java index 1ef3b5db1e9..d2c8e8a342f 100644 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestLogFileTask.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestLogFileTask.java @@ -20,10 +20,9 @@ import org.apache.inlong.agent.common.AgentThreadFactory; import org.apache.inlong.agent.conf.TaskProfile; import org.apache.inlong.agent.constant.CycleUnitType; -import org.apache.inlong.agent.constant.TaskConstants; import org.apache.inlong.agent.core.task.TaskManager; import org.apache.inlong.agent.plugin.AgentBaseTestsHelper; -import org.apache.inlong.agent.plugin.task.file.LogFileTask; +import org.apache.inlong.agent.plugin.task.logcollection.local.FileTask; import org.apache.inlong.common.enums.TaskStateEnum; import org.junit.AfterClass; @@ -40,10 +39,8 @@ import org.slf4j.LoggerFactory; import java.io.File; -import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Arrays; -import java.util.Date; import java.util.List; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; @@ -52,7 +49,7 @@ import static org.awaitility.Awaitility.await; @RunWith(PowerMockRunner.class) -@PrepareForTest(LogFileTask.class) +@PrepareForTest(FileTask.class) @PowerMockIgnore({"javax.management.*"}) public class TestLogFileTask { @@ -83,17 +80,17 @@ public static void teardown() throws Exception { public void testScan() throws Exception { doTest(1, Arrays.asList("testScan/20230928_1/test_1.txt"), resourceParentPath + "/YYYYMMDD_[0-9]+/test_[0-9]+.txt", CycleUnitType.DAY, Arrays.asList("20230928"), - "2023-09-28 00:00:00", "2023-09-30 23:00:00"); + "20230928", "20230930"); doTest(2, Arrays.asList("testScan/2023092810_1/test_1.txt"), resourceParentPath + "/YYYYMMDDhh_[0-9]+/test_[0-9]+.txt", - CycleUnitType.HOUR, Arrays.asList("2023092810"), "2023-09-28 00:00:00", "2023-09-30 23:00:00"); + CycleUnitType.HOUR, Arrays.asList("2023092810"), "2023092800", "2023093023"); doTest(3, Arrays.asList("testScan/202309281030_1/test_1.txt", "testScan/202309301059_1/test_1.txt"), resourceParentPath + "/YYYYMMDDhhmm_[0-9]+/test_[0-9]+.txt", - CycleUnitType.MINUTE, Arrays.asList("202309281030", "202309301059"), "2023-09-28 00:00:00", - "2023-09-30 23:00:00"); + CycleUnitType.MINUTE, Arrays.asList("202309281030", "202309301059"), "202309280000", + "202309302300"); doTest(4, Arrays.asList("testScan/20241030/23/59.txt"), resourceParentPath + "/YYYYMMDD/hh/mm.txt", - CycleUnitType.MINUTE, Arrays.asList("202410302359"), "2024-10-30 00:00:00", "2024-10-31 00:00:00"); + CycleUnitType.MINUTE, Arrays.asList("202410302359"), "202410300000", "202410310000"); } private void doTest(int taskId, List resources, String pattern, String cycle, List srcDataTimes, @@ -103,21 +100,14 @@ private void doTest(int taskId, List resources, String pattern, String c for (int i = 0; i < resources.size(); i++) { resourceName.add(LOADER.getResource(resources.get(i)).getPath()); } - TaskProfile taskProfile = helper.getTaskProfile(taskId, pattern, "csv", true, 0L, 0L, TaskStateEnum.RUNNING, - cycle, - "GMT+8:00", null); - LogFileTask dayTask = null; + TaskProfile taskProfile = + helper.getFileTaskProfile(taskId, pattern, "csv", true, startTime, endTime, TaskStateEnum.RUNNING, + cycle, "GMT+8:00", null); + FileTask dayTask = null; final List fileName = new ArrayList(); final List dataTime = new ArrayList(); try { - - Date parse = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").parse(startTime); - long start = parse.getTime(); - parse = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").parse(endTime); - long end = parse.getTime(); - taskProfile.setLong(TaskConstants.TASK_START_TIME, start); - taskProfile.setLong(TaskConstants.TASK_END_TIME, end); - dayTask = PowerMockito.spy(new LogFileTask()); + dayTask = PowerMockito.spy(new FileTask()); PowerMockito.doAnswer(invocation -> { fileName.add(invocation.getArgument(0)); dataTime.add(invocation.getArgument(1)); @@ -128,7 +118,7 @@ private void doTest(int taskId, List resources, String pattern, String c dayTask.init(manager, taskProfile, manager.getInstanceBasicStore()); EXECUTOR_SERVICE.submit(dayTask); } catch (Exception e) { - LOGGER.error("source init error {}", e); + LOGGER.error("source init error", e); Assert.assertTrue("source init error", false); } await().atMost(10, TimeUnit.SECONDS) diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestTaskManager.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestTaskManager.java index 608d3adec67..c6ba4a449bf 100755 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestTaskManager.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/task/TestTaskManager.java @@ -22,6 +22,7 @@ import org.apache.inlong.agent.plugin.AgentBaseTestsHelper; import org.apache.inlong.agent.store.TaskStore; import org.apache.inlong.common.enums.TaskStateEnum; +import org.apache.inlong.common.enums.TaskTypeEnum; import org.junit.AfterClass; import org.junit.Assert; @@ -34,6 +35,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import static org.apache.inlong.agent.constant.TaskConstants.TASK_TYPE; import static org.awaitility.Awaitility.await; public class TestTaskManager { @@ -58,9 +60,10 @@ public void testTaskManager() { manager = new TaskManager(); TaskStore taskStore = manager.getTaskStore(); for (int i = 1; i <= 10; i++) { - TaskProfile taskProfile = helper.getTaskProfile(i, pattern, "csv", false, 0L, 0L, TaskStateEnum.RUNNING, - "D", "GMT+8:00", null); - taskProfile.setTaskClass(MockTask.class.getCanonicalName()); + TaskProfile taskProfile = + helper.getFileTaskProfile(i, pattern, "csv", false, "", "", TaskStateEnum.RUNNING, + "D", "GMT+8:00", null); + taskProfile.setInt(TASK_TYPE, TaskTypeEnum.MOCK.getType()); taskStore.storeTask(taskProfile); } manager.start(); @@ -74,10 +77,10 @@ public void testTaskManager() { Assert.assertTrue("manager start error", false); } - TaskProfile taskProfile1 = helper.getTaskProfile(100, pattern, "csv", false, 0L, 0L, TaskStateEnum.RUNNING, + TaskProfile taskProfile1 = helper.getFileTaskProfile(100, pattern, "csv", false, "", "", TaskStateEnum.RUNNING, "D", "GMT+8:00", null); String taskId1 = taskProfile1.getTaskId(); - taskProfile1.setTaskClass(MockTask.class.getCanonicalName()); + taskProfile1.setInt(TASK_TYPE, TaskTypeEnum.MOCK.getType()); List taskProfiles1 = new ArrayList<>(); taskProfiles1.add(taskProfile1); // test add @@ -99,9 +102,9 @@ public void testTaskManager() { Assert.assertTrue(manager.getTaskProfile(taskId1).getState() == TaskStateEnum.RUNNING); // test delete - TaskProfile taskProfile2 = helper.getTaskProfile(200, pattern, "csv", false, 0L, 0L, TaskStateEnum.RUNNING, + TaskProfile taskProfile2 = helper.getFileTaskProfile(200, pattern, "csv", false, "", "", TaskStateEnum.RUNNING, "D", "GMT+8:00", null); - taskProfile2.setTaskClass(MockTask.class.getCanonicalName()); + taskProfile2.setInt(TASK_TYPE, TaskTypeEnum.MOCK.getType()); List taskProfiles2 = new ArrayList<>(); taskProfiles2.add(taskProfile2); manager.submitTaskProfiles(taskProfiles2); diff --git a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/utils/TestUtils.java b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/utils/TestUtils.java index cf1b7c8f958..a1f13122e30 100644 --- a/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/utils/TestUtils.java +++ b/inlong-agent/agent-plugins/src/test/java/org/apache/inlong/agent/plugin/utils/TestUtils.java @@ -17,8 +17,8 @@ package org.apache.inlong.agent.plugin.utils; -import org.apache.inlong.agent.plugin.utils.file.FilePathUtil; -import org.apache.inlong.agent.plugin.utils.file.NewDateUtils; +import org.apache.inlong.agent.plugin.utils.regex.NewDateUtils; +import org.apache.inlong.agent.plugin.utils.regex.PatternUtil; import org.apache.inlong.agent.utils.DateTransUtils; import org.apache.inlong.common.metric.MetricRegister; @@ -91,6 +91,8 @@ public void testPattern() throws ParseException { Arrays.asList("/data/log_minute", "minute_YYYYMMDDhh*", "mm.log_[0-9]+")); testCutDirectoryByWildcard("/data/123+/YYYYMMDDhhmm.log", Arrays.asList("/data", "123+", "YYYYMMDDhhmm.log")); + testCutDirectoryByWildcard("/data/2024112610*/test.log", + Arrays.asList("/data", "2024112610*", "test.log")); /* * 1 cut the file name 2 cut the path contains wildcard or date expression @@ -103,6 +105,20 @@ public void testPattern() throws ParseException { Arrays.asList("/data", "123*/MMDD", "test.log")); testCutDirectoryByWildcardAndDateExpression("/data/YYYYMMDD/123*/test.log", Arrays.asList("/data", "YYYYMMDD/123*", "test.log")); + + /* + * get the string before the first wildcard + */ + testGetBeforeFirstWildcard("/data/YYYYMM/YYYaaMM/YYYYMMDDhhmm.log", + "/data/YYYYMM/YYYaaMM/YYYYMMDDhhmm"); + testGetBeforeFirstWildcard("/data/123*/MMDD/test.log", + "/data/123"); + testGetBeforeFirstWildcard("/data/YYYYMMDD/123*/test.log", + "/data/YYYYMMDD/123"); + testGetBeforeFirstWildcard("test/65535_YYYYMMDD_hh00.log", + "test/65535_YYYYMMDD_hh00"); + testGetBeforeFirstWildcard("/data/YYYYMMDD/*123/test.log", + "/data/YYYYMMDD/"); } private void testReplaceDateExpression(String src, String dst) throws ParseException { @@ -113,12 +129,17 @@ private void testReplaceDateExpression(String src, String dst) throws ParseExcep } private void testCutDirectoryByWildcard(String src, List dst) { - ArrayList directories = FilePathUtil.cutDirectoryByWildcard(src); + ArrayList directories = PatternUtil.cutDirectoryByWildcard(src); Assert.assertEquals(directories, dst); } + private void testGetBeforeFirstWildcard(String src, String dst) { + String temp = PatternUtil.getBeforeFirstWildcard(src); + Assert.assertEquals(dst, temp); + } + private void testCutDirectoryByWildcardAndDateExpression(String src, List dst) { - ArrayList directoryLayers = FilePathUtil.cutDirectoryByWildcardAndDateExpression(src); + ArrayList directoryLayers = PatternUtil.cutDirectoryByWildcardAndDateExpression(src); Assert.assertEquals(directoryLayers, dst); } diff --git a/inlong-agent/bin/agent.sh b/inlong-agent/bin/agent.sh index 9d5bc866863..5db202394fb 100755 --- a/inlong-agent/bin/agent.sh +++ b/inlong-agent/bin/agent.sh @@ -37,13 +37,28 @@ function help() { echo " help: get help from inlong agent" } + +function getPid() { + local process_name="inlong-agent" + local user=$(whoami) + local pid=$(ps -u $user -f | grep 'java' | grep "$process_name" | grep -v grep | awk '{print $2}') + + if [ -z "$pid" ]; then + echo "No matching process found." + return 1 + fi + + echo "$pid" + return 0 +} + function running() { - process=$("$JPS" | grep "AgentMain" | grep -v grep) - if [ "${process}" = "" ]; then - return 1; - else - return 0; - fi + pid=$(getPid) + if [ $? -eq 0 ]; then + return 0 + else + return 1 + fi } # start agent @@ -53,9 +68,9 @@ function start_agent() { exit 1 fi if [ "$ENABLE_OBSERVABILITY" = "true" ]; then - nohup ${JAVA} ${AGENT_ARGS} -javaagent:${OTEL_AGENT} org.apache.inlong.agent.core.AgentMain > /dev/null 2>&1 & + nohup ${JAVA} ${AGENT_ARGS} ${arg_uniq} -javaagent:${OTEL_AGENT} org.apache.inlong.agent.core.AgentMain > /dev/null 2>&1 & else - nohup ${JAVA} ${AGENT_ARGS} org.apache.inlong.agent.core.AgentMain > /dev/null 2>&1 & + nohup ${JAVA} ${AGENT_ARGS} ${arg_uniq} org.apache.inlong.agent.core.AgentMain > /dev/null 2>&1 & fi } @@ -66,7 +81,7 @@ function stop_agent() { exit 1 fi count=0 - pid=$("$JPS" | grep "AgentMain" | grep -v grep | awk '{print $1}') + pid=$(getPid) while running; do (( count++ )) diff --git a/inlong-common/src/main/java/org/apache/inlong/common/constant/SinkType.java b/inlong-common/src/main/java/org/apache/inlong/common/constant/SinkType.java index c2816baf1e7..f3eb862d87b 100644 --- a/inlong-common/src/main/java/org/apache/inlong/common/constant/SinkType.java +++ b/inlong-common/src/main/java/org/apache/inlong/common/constant/SinkType.java @@ -19,6 +19,8 @@ public class SinkType { + public static final String ICEBERG = "ICEBERG"; + public static final String HIVE = "HIVE"; public static final String KAFKA = "KAFKA"; public static final String PULSAR = "PULSAR"; public static final String CLS = "CLS"; diff --git a/inlong-common/src/main/java/org/apache/inlong/common/enums/TaskTypeEnum.java b/inlong-common/src/main/java/org/apache/inlong/common/enums/TaskTypeEnum.java index c84ea142dbd..3a032e6511e 100644 --- a/inlong-common/src/main/java/org/apache/inlong/common/enums/TaskTypeEnum.java +++ b/inlong-common/src/main/java/org/apache/inlong/common/enums/TaskTypeEnum.java @@ -44,11 +44,9 @@ public enum TaskTypeEnum { REDIS(11), MQTT(12), HUDI(13), - + COS(14), // only used for unit test - MOCK(201) - - ; + MOCK(201); private static final Map TASK_TYPE_ENUM_MAP = Maps.newHashMap(); diff --git a/inlong-common/src/main/java/org/apache/inlong/common/pojo/dataproxy/DataProxyNodeResponse.java b/inlong-common/src/main/java/org/apache/inlong/common/pojo/dataproxy/DataProxyNodeResponse.java index 0d7237fdfeb..a48219a930a 100644 --- a/inlong-common/src/main/java/org/apache/inlong/common/pojo/dataproxy/DataProxyNodeResponse.java +++ b/inlong-common/src/main/java/org/apache/inlong/common/pojo/dataproxy/DataProxyNodeResponse.java @@ -30,7 +30,6 @@ public class DataProxyNodeResponse { /** * DataProxy cluster id */ - @Deprecated private Integer clusterId; private String reportSourceType; diff --git a/inlong-dashboard/src/plugins/sync/common/SyncDefaultInfo.ts b/inlong-dashboard/src/plugins/sync/common/SyncDefaultInfo.ts index 7bf18927d9a..7e8456d7490 100644 --- a/inlong-dashboard/src/plugins/sync/common/SyncDefaultInfo.ts +++ b/inlong-dashboard/src/plugins/sync/common/SyncDefaultInfo.ts @@ -151,7 +151,6 @@ export class SyncDefaultInfo implements DataWithBackend, RenderRow, RenderList { rules: [{ required: true }], props: values => { return { - disabled: values.scheduleEngine === 'DolphinScheduler', options: [ { label: i18n.t('meta.Synchronize.Conventional'), diff --git a/inlong-dashboard/src/ui/locales/cn.json b/inlong-dashboard/src/ui/locales/cn.json index 535063ee005..4f631bb1518 100644 --- a/inlong-dashboard/src/ui/locales/cn.json +++ b/inlong-dashboard/src/ui/locales/cn.json @@ -1,5 +1,6 @@ { "basic.Edit": "编辑", + "basic.Search": "搜索", "basic.Detail": "详情", "basic.Operating": "操作", "basic.OperatingSuccess": "操作成功", @@ -1020,5 +1021,35 @@ "pages.GroupDataTemplate.VisibleRange.InCharges":"责任人", "pages.GroupDataTemplate.VisibleRange.Tenant":"租户", "miscellaneous.total": "... 共", - "miscellaneous.tenants": "个租户" + "miscellaneous.tenants": "个租户", + "meta.Sinks.DirtyData.DirtyDataPartition": "脏数据分区", + "meta.Sinks.DirtyData.DataFlowId": "数据目标Id", + "meta.Sinks.DirtyData.GroupId": "数据组Id", + "meta.Sinks.DirtyData.StreamId": "数据流Id", + "meta.Sinks.DirtyData.ReportTime": "上报时间", + "meta.Sinks.DirtyData.DataTime": "数据时间", + "meta.Sinks.DirtyData.ServerType": "服务类型", + "meta.Sinks.DirtyData.DirtyType": "脏数据类型", + "meta.Sinks.DirtyData.DirtyMessage": "脏数据信息", + "meta.Sinks.DirtyData.ExtInfo": "额外信息", + "meta.Sinks.DirtyData.DirtyData": "脏数据", + "meta.Sinks.DirtyData.DirtyDetailWarning": "脏数据任务正在运行,请稍后再试", + "meta.Sinks.DirtyData.DirtyTrendWarning": "脏数据趋势任务正在运行,请稍后再试", + "meta.Sinks.DirtyData.DataCount": "脏数据条数", + "meta.Sinks.DirtyData.Search.DirtyType": "脏数据类型", + "meta.Sinks.DirtyData.Search.ServerType": "服务类型", + "meta.Sinks.DirtyData.StartTimeError": "开始时间不能大于当前时间", + "meta.Sinks.DirtyData.endTimeNotGreaterThanStartTime": "结束时间不能大于当前时间", + "meta.Sinks.DirtyData.TimeIntervalError": "时间间隔不能超过七天", + "meta.Sinks.DirtyTrend.DataTimeUnit":"时间单位", + "meta.Sinks.DirtyTrend.Day":"天", + "meta.Sinks.DirtyTrend.Hour":"小时", + "meta.Sinks.DirtyData.Detail":"详情", + "meta.Sinks.DirtyData.Trend":"趋势", + "meta.Sinks.DirtyData":"脏数据查询", + "meta.Sinks.DirtyData.DirtyType.DeserializeError":"反序列化错误", + "meta.Sinks.DirtyData.DirtyType.FieldMappingError":"字段映射错误", + "meta.Sinks.DirtyData.DirtyType.LoadError":"加载错误", + "meta.Sinks.DirtyData.Search.KeyWordHelp":"请输入关键字", + "meta.Sinks.DirtyData.Search.KeyWord":"关键字" } diff --git a/inlong-dashboard/src/ui/locales/en.json b/inlong-dashboard/src/ui/locales/en.json index b74f54e58e1..b7e60d316bd 100644 --- a/inlong-dashboard/src/ui/locales/en.json +++ b/inlong-dashboard/src/ui/locales/en.json @@ -1,5 +1,6 @@ { "basic.Edit": "Edit", + "basic.Search": "Search", "basic.Detail": "Detail", "basic.Operating": "Operation", "basic.OperatingSuccess": "Operating success", @@ -374,6 +375,7 @@ "meta.Sinks.Cls.Tag": "Tag", "meta.Sinks.Cls.Tokenizer": "Tokenizer rule", "meta.Sinks.Cls.IsMetaField": "Is meta field", + "meta.Group.InlongGroupId": "Inlong group id", "meta.Group.InlongGroupIdRules": "Only English letters, numbers, dots(.), minus(-), and underscores(_)", "meta.Group.InlongGroupName": "Inlong group name", @@ -1020,5 +1022,35 @@ "pages.GroupDataTemplate.VisibleRange.InCharges":"Owner", "pages.GroupDataTemplate.VisibleRange.Tenant":"Tenant", "miscellaneous.total": "... total ", - "miscellaneous.tenants": " tenants" + "miscellaneous.tenants": " tenants", + "meta.Sinks.DirtyData.DirtyDataPartition": "Dirty Data Partition", + "meta.Sinks.DirtyData.DataFlowId": "Data Flow Id", + "meta.Sinks.DirtyData.GroupId": "Group Id", + "meta.Sinks.DirtyData.StreamId": "Stream Id", + "meta.Sinks.DirtyData.ReportTime": "Report Time", + "meta.Sinks.DirtyData.DataTime": "Data Time", + "meta.Sinks.DirtyData.ServerType": "Server Type", + "meta.Sinks.DirtyData.DirtyType": "Dirty Type", + "meta.Sinks.DirtyData.DirtyMessage": "Dirty Message", + "meta.Sinks.DirtyData.ExtInfo": "Ext Info", + "meta.Sinks.DirtyData.DirtyData": "Dirty Data", + "meta.Sinks.DirtyData.DirtyDetailWarning": "The dirty data task is running, please try again later", + "meta.Sinks.DirtyData.DirtyTrendWarning": "The dirty data trending task is running, try again later", + "meta.Sinks.DirtyData.DataCount": "Data Count", + "meta.Sinks.DirtyData.Search.DirtyType": "Dirty Type", + "meta.Sinks.DirtyData.Search.ServerType": "Server Type", + "meta.Sinks.DirtyData.StartTimeError": "The start time cannot be greater than the current time", + "meta.Sinks.DirtyData.endTimeNotGreaterThanStartTime": "The end time cannot be greater than the current time", + "meta.Sinks.DirtyData.TimeIntervalError": "The time interval cannot be more than seven days", + "meta.Sinks.DirtyTrend.DataTimeUnit":"DataTime Unit", + "meta.Sinks.DirtyTrend.Day":"Day", + "meta.Sinks.DirtyTrend.Hour":"Hour", + "meta.Sinks.DirtyData.Detail":"Detail", + "meta.Sinks.DirtyData.Trend":"Trend", + "meta.Sinks.DirtyData":"Dirty Data Query", + "meta.Sinks.DirtyData.DirtyType.DeserializeError":"Deserialize Error", + "meta.Sinks.DirtyData.DirtyType.FieldMappingError":"Field Mapping Error", + "meta.Sinks.DirtyData.DirtyType.LoadError":"Load Error", + "meta.Sinks.DirtyData.Search.KeyWordHelp":"Please enter a keyword", + "meta.Sinks.DirtyData.Search.KeyWord":"Key word" } diff --git a/inlong-dashboard/src/ui/pages/AgentModule/CreateModal.tsx b/inlong-dashboard/src/ui/pages/AgentModule/CreateModal.tsx index 68adb5d06cc..09d5b77e8d3 100644 --- a/inlong-dashboard/src/ui/pages/AgentModule/CreateModal.tsx +++ b/inlong-dashboard/src/ui/pages/AgentModule/CreateModal.tsx @@ -91,7 +91,10 @@ const Comp: React.FC = ({ id, type, ...modalProps }) => { type: 'textarea', label: i18n.t('pages.ModuleAgent.Config.CheckCommand'), name: 'checkCommand', - initialValue: "ps aux | grep core.AgentMain | grep java | grep -v grep | awk '{print $2}'", + initialValue: + type === 'INSTALLER' + ? 'echo "installer"' + : "ps aux | grep core.AgentMain | grep java | grep -v grep | awk '{print $2}'", props: { showCount: true, maxLength: 1000, @@ -102,7 +105,9 @@ const Comp: React.FC = ({ id, type, ...modalProps }) => { label: i18n.t('pages.ModuleAgent.Config.InstallCommand'), name: 'installCommand', initialValue: - 'cd ~/inlong/inlong-agent/bin;sh agent.sh stop;rm -rf ~/inlong/inlong-agent-temp;mkdir -p ~/inlong/inlong-agent-temp;cp -r ~/inlong/inlong-agent/.localdb ', + type === 'INSTALLER' + ? 'echo ""' + : 'cd ~/inlong/inlong-agent/bin;sh agent.sh stop;rm -rf ~/inlong/inlong-agent-temp;mkdir -p ~/inlong/inlong-agent-temp;cp -r ~/inlong/inlong-agent/.localdb ', props: { showCount: true, maxLength: 1000, @@ -112,7 +117,8 @@ const Comp: React.FC = ({ id, type, ...modalProps }) => { type: 'textarea', label: i18n.t('pages.ModuleAgent.Config.StartCommand'), name: 'startCommand', - initialValue: 'cd ~/inlong/inlong-agent/bin;sh agent.sh start', + initialValue: + type === 'INSTALLER' ? 'echo ""' : 'cd ~/inlong/inlong-agent/bin;sh agent.sh start', props: { showCount: true, maxLength: 1000, @@ -122,7 +128,8 @@ const Comp: React.FC = ({ id, type, ...modalProps }) => { type: 'textarea', label: i18n.t('pages.ModuleAgent.Config.StopCommand'), name: 'stopCommand', - initialValue: 'cd ~/inlong/inlong-agent/bin;sh agent.sh stop', + initialValue: + type === 'INSTALLER' ? 'echo ""' : 'cd ~/inlong/inlong-agent/bin;sh agent.sh stop', props: { showCount: true, maxLength: 1000, @@ -132,7 +139,7 @@ const Comp: React.FC = ({ id, type, ...modalProps }) => { type: 'textarea', label: i18n.t('pages.ModuleAgent.Config.UninstallCommand'), name: 'uninstallCommand', - initialValue: 'echo empty uninstall cmd', + initialValue: type === 'INSTALLER' ? 'echo ""' : 'echo empty uninstall cmd', props: { showCount: true, maxLength: 1000, diff --git a/inlong-dashboard/src/ui/pages/GroupDetail/DataStorage/index.tsx b/inlong-dashboard/src/ui/pages/GroupDetail/DataStorage/index.tsx index b91cac850d2..5f1086d91e6 100644 --- a/inlong-dashboard/src/ui/pages/GroupDetail/DataStorage/index.tsx +++ b/inlong-dashboard/src/ui/pages/GroupDetail/DataStorage/index.tsx @@ -25,6 +25,7 @@ import { TableOutlined, EditOutlined, DeleteOutlined, + AreaChartOutlined, } from '@ant-design/icons'; import HighTable from '@/ui/components/HighTable'; import { defaultSize } from '@/configs/pagination'; @@ -36,6 +37,7 @@ import request from '@/core/utils/request'; import { pickObjectArray } from '@/core/utils'; import { CommonInterface } from '../common'; import { sinks } from '@/plugins/sinks'; +import DirtyModal from '@/ui/pages/common/DirtyModal'; interface Props extends CommonInterface { inlongStreamId?: string; @@ -58,7 +60,12 @@ const Comp = ({ inlongGroupId, inlongStreamId, readonly }: Props, ref) => { const [createModal, setCreateModal] = useState>({ open: false, }); - + const [dirtyModal, setDirtyModal] = useState>({ + open: false, + }); + const onOpenDirtyModal = useCallback(({ id }) => { + setDirtyModal({ open: true, id }); + }, []); const { data, loading, @@ -173,6 +180,9 @@ const Comp = ({ inlongGroupId, inlongStreamId, readonly }: Props, ref) => { + ), } as any, @@ -238,6 +248,9 @@ const Comp = ({ inlongGroupId, inlongStreamId, readonly }: Props, ref) => { , + , ]} > @@ -277,6 +290,15 @@ const Comp = ({ inlongGroupId, inlongStreamId, readonly }: Props, ref) => { }} onCancel={() => setCreateModal({ open: false })} /> + { + await getList(); + setDirtyModal({ open: false }); + }} + onCancel={() => setDirtyModal({ open: false })} + /> ); }; diff --git a/inlong-dashboard/src/ui/pages/SynchronizeDetail/Info/index.tsx b/inlong-dashboard/src/ui/pages/SynchronizeDetail/Info/index.tsx index f5ed2767617..c6ccdd0a828 100644 --- a/inlong-dashboard/src/ui/pages/SynchronizeDetail/Info/index.tsx +++ b/inlong-dashboard/src/ui/pages/SynchronizeDetail/Info/index.tsx @@ -229,10 +229,6 @@ const Comp = ({ inlongGroupId, inlongStreamId, readonly, isCreate }: Props, ref) initialValues={data} onValuesChange={(c, values) => { setMqType(values.mqType); - if (c.scheduleEngine === 'DolphinScheduler') { - form.setFieldsValue({ scheduleType: 1 }); - values.scheduleType = 1; - } }} useMaxWidth={1400} col={14} diff --git a/inlong-dashboard/src/ui/pages/SynchronizeDetail/SyncSink/index.tsx b/inlong-dashboard/src/ui/pages/SynchronizeDetail/SyncSink/index.tsx index 9e7c243630f..2d5f2a761ae 100644 --- a/inlong-dashboard/src/ui/pages/SynchronizeDetail/SyncSink/index.tsx +++ b/inlong-dashboard/src/ui/pages/SynchronizeDetail/SyncSink/index.tsx @@ -25,6 +25,7 @@ import { TableOutlined, EditOutlined, DeleteOutlined, + AreaChartOutlined, } from '@ant-design/icons'; import HighTable from '@/ui/components/HighTable'; import { defaultSize } from '@/configs/pagination'; @@ -36,6 +37,7 @@ import request from '@/core/utils/request'; import { pickObjectArray } from '@/core/utils'; import { sinks } from '@/plugins/sinks'; import { CommonInterface } from '../common'; +import DirtyModal from '@/ui/pages/common/DirtyModal'; interface Props extends CommonInterface { inlongStreamId: string; @@ -58,7 +60,12 @@ const Comp = ({ inlongGroupId, inlongStreamId, sinkMultipleEnable, readonly }: P const [createModal, setCreateModal] = useState>({ open: false, }); - + const [dirtyModal, setDirtyModal] = useState>({ + open: false, + }); + const onOpenDirtyModal = useCallback(({ id }) => { + setDirtyModal({ open: true, id }); + }, []); const { data, loading, @@ -179,6 +186,9 @@ const Comp = ({ inlongGroupId, inlongStreamId, sinkMultipleEnable, readonly }: P + ), } as any, @@ -249,6 +259,9 @@ const Comp = ({ inlongGroupId, inlongStreamId, sinkMultipleEnable, readonly }: P , + , ]} > @@ -289,6 +302,15 @@ const Comp = ({ inlongGroupId, inlongStreamId, sinkMultipleEnable, readonly }: P }} onCancel={() => setCreateModal({ open: false })} /> + { + await getList(); + setDirtyModal({ open: false }); + }} + onCancel={() => setDirtyModal({ open: false })} + /> ); }; diff --git a/inlong-dashboard/src/ui/pages/common/DirtyModal/conf.tsx b/inlong-dashboard/src/ui/pages/common/DirtyModal/conf.tsx new file mode 100644 index 00000000000..ce18eceb8a7 --- /dev/null +++ b/inlong-dashboard/src/ui/pages/common/DirtyModal/conf.tsx @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import i18n from '@/i18n'; + +export const statusList = [ + { + label: i18n.t('meta.Sinks.DirtyData.DirtyType.DeserializeError'), + value: 'DeserializeError', + }, + { + label: i18n.t('meta.Sinks.DirtyData.DirtyType.FieldMappingError'), + value: 'FieldMappingError', + }, + { + label: i18n.t('meta.Sinks.DirtyData.DirtyType.LoadError'), + value: 'LoadError', + }, +]; + +export const statusMap = statusList.reduce( + (acc, cur) => ({ + ...acc, + [cur.value]: cur, + }), + {}, +); + +export const genStatusTag = value => { + return statusMap[value]; +}; diff --git a/inlong-dashboard/src/ui/pages/common/DirtyModal/index.tsx b/inlong-dashboard/src/ui/pages/common/DirtyModal/index.tsx new file mode 100644 index 00000000000..402e6dbe7e1 --- /dev/null +++ b/inlong-dashboard/src/ui/pages/common/DirtyModal/index.tsx @@ -0,0 +1,595 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import React, { useEffect, useState } from 'react'; +import { Button, message, Modal, Tabs, TabsProps } from 'antd'; +import { ModalProps } from 'antd/es/modal'; +import i18n from '@/i18n'; +import HighTable from '@/ui/components/HighTable'; + +import dayjs from 'dayjs'; +import request from '@/core/utils/request'; +import { useForm } from 'antd/es/form/Form'; +import FormGenerator from '@/ui/components/FormGenerator'; +import Charts from '@/ui/components/Charts'; +import { genStatusTag } from '@/ui/pages/common/DirtyModal/conf'; + +export interface Props extends ModalProps { + id?: number; +} +const Comp: React.FC = ({ ...modalProps }) => { + const [form1] = useForm(); + const [form2] = useForm(); + const [loading, setLoading] = useState(false); + const getColumns = [ + { + title: i18n.t('meta.Sinks.DirtyData.DataFlowId'), + dataIndex: 'dataFlowId', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.GroupId'), + dataIndex: 'groupId', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.StreamId'), + dataIndex: 'streamId', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.ReportTime'), + dataIndex: 'reportTime', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.DataTime'), + dataIndex: 'dataTime', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.ServerType'), + dataIndex: 'serverType', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.DirtyType'), + dataIndex: 'dirtyType', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.DirtyMessage'), + dataIndex: 'dirtyMessage', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.ExtInfo'), + dataIndex: 'extInfo', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.DirtyData'), + dataIndex: 'dirtyData', + width: 90, + }, + { + title: i18n.t('meta.Sinks.DirtyData.DirtyDataPartition'), + dataIndex: 'dirtyDataPartition', + width: 90, + }, + ]; + + const defaultDetailOptions = { + keyword: '', + dataCount: 10, + dirtyType: '', + serverType: '', + startTime: dayjs().format('YYYYMMDDHH'), + endTime: dayjs().format('YYYYMMDDHH'), + }; + const defaultTrendOptions = { + dataTimeUnit: 'D', + dirtyType: '', + serverType: '', + startTime: dayjs().format('YYYYMMDD'), + endTime: dayjs().format('YYYYMMDD'), + }; + const [options, setOptions] = useState(defaultDetailOptions); + const [trendOptions, setTrendOption] = useState(defaultTrendOptions); + const [data, setData] = useState([]); + const [trendData, setTrendData] = useState([]); + const [tabValue, setTabValue] = useState('detail'); + useEffect(() => { + if (modalProps.open) { + if (tabValue === 'detail') { + setOptions(defaultDetailOptions); + form1.resetFields(); + getTaskResult().then(item => { + setData(item); + }); + } + if (tabValue === 'trend') { + setTrendOption(defaultTrendOptions); + form2.resetFields(); + form2.setFieldsValue({ + dataTimeUnit: 'D', + }); + getTrendData().then(item => { + setTrendData(item); + }); + } + } + }, [modalProps.open, tabValue]); + const [messageApi, contextHolder] = message.useMessage(); + const warning = () => { + messageApi.open({ + type: 'warning', + content: + tabValue === 'detail' + ? i18n.t('meta.Sinks.DirtyData.DirtyDetailWarning') + : i18n.t('meta.Sinks.DirtyData.DirtyTrendWarning'), + }); + }; + const getTaskResult = async () => { + setLoading(true); + const taskId = await getTaskId(); + const status = await request({ + url: '/sink/SqlTaskStatus/' + taskId, + method: 'GET', + }); + if (status === 'success') { + const data = await request({ + url: '/sink/getDirtyData/' + taskId, + method: 'GET', + }); + setLoading(false); + return data; + } else { + setLoading(false); + warning(); + } + return []; + }; + const getTaskId = async () => { + const data = await request({ + url: '/sink/listDirtyData', + method: 'POST', + data: { + ...options, + startTime: options.startTime ? dayjs(options.startTime).format('YYYYMMDDHH') : '', + endTime: options.endTime ? dayjs(options.endTime).format('YYYYMMDDHH') : '', + dataCount: form1.getFieldValue('dataCount') || 10, + keyword: form1.getFieldValue('keyword') || '', + sinkIdList: [modalProps.id], + }, + }); + return data.taskId; + }; + + const getTrendData = async () => { + const taskId = await getTrendTaskId(); + const status = await request({ + url: '/sink/SqlTaskStatus/' + taskId, + method: 'GET', + }); + + if (status === 'success') { + const data = await request({ + url: '/sink/getDirtyDataTrend/' + taskId, + method: 'GET', + }); + return data; + } else { + warning(); + } + return []; + }; + + const getTrendTaskId = async () => { + const data = await request({ + url: '/sink/listDirtyDataTrend', + method: 'POST', + data: { + ...trendOptions, + sinkIdList: [modalProps.id], + }, + }); + return data.taskId; + }; + const onSearch = async () => { + await form1.validateFields(); + await getTaskResult().then(item => { + setData(item); + }); + }; + const onTrendSearch = async () => { + await form2.validateFields(); + await getTrendData().then(item => { + setTrendData(item); + }); + }; + + const getDetailFilterFormContent = defaultValues => [ + { + type: 'input', + label: i18n.t('meta.Sinks.DirtyData.Search.KeyWord'), + name: 'keyword', + props: { + placeholder: i18n.t('meta.Sinks.DirtyData.Search.KeyWordHelp'), + }, + }, + { + label: i18n.t('meta.Sinks.DirtyData.DataCount'), + type: 'inputnumber', + name: 'dataCount', + }, + { + label: i18n.t('meta.Sinks.DirtyData.Search.DirtyType'), + type: 'select', + name: 'dirtyType', + props: { + allowClear: true, + options: [ + { + label: i18n.t('meta.Sinks.DirtyData.DirtyType.DeserializeError'), + value: 'DeserializeError', + }, + { + label: i18n.t('meta.Sinks.DirtyData.DirtyType.FieldMappingError'), + value: 'FieldMappingError', + }, + { + label: i18n.t('meta.Sinks.DirtyData.DirtyType.LoadError'), + value: 'LoadError', + }, + ], + }, + }, + { + label: i18n.t('meta.Sinks.DirtyData.Search.ServerType'), + type: 'select', + name: 'serverType', + props: { + allowClear: true, + options: [ + { + label: 'TubeMQ', + value: 'TubeMQ', + }, + { + label: 'Iceberg', + value: 'Iceberg', + }, + ], + }, + }, + { + type: 'datepicker', + label: i18n.t('pages.GroupDetail.Audit.StartDate'), + name: 'startTime', + initialValue: dayjs(options.startTime), + props: { + allowClear: true, + showTime: true, + format: 'YYYY-MM-DD HH', + }, + rules: [ + { required: true }, + ({ getFieldValue }) => ({ + validator(_, value) { + if (Boolean(value)) { + if (value.isAfter(dayjs())) { + return Promise.reject(new Error(i18n.t('meta.Sinks.DirtyData.StartTimeError'))); + } + } + return Promise.resolve(); + }, + }), + ], + }, + { + type: 'datepicker', + label: i18n.t('pages.GroupDetail.Audit.EndDate'), + name: 'endTime', + initialValue: dayjs(options.endTime), + props: values => { + return { + allowClear: true, + showTime: true, + format: 'YYYY-MM-DD HH', + }; + }, + rules: [ + { required: true }, + ({ getFieldValue }) => ({ + validator(_, value) { + if (Boolean(value)) { + if (value.isAfter(dayjs())) { + return Promise.reject(new Error(i18n.t('endTimeNotGreaterThanStartTime'))); + } + const timeDiff = value.diff(getFieldValue('startDate'), 'day'); + if (timeDiff <= 7) { + return Promise.resolve(); + } + return Promise.reject(new Error(i18n.t('meta.Sinks.DirtyData.TimeIntervalError'))); + } + return Promise.resolve(); + }, + }), + ], + }, + { + type: ( + + ), + }, + ]; + const getTendFilterFormContent = defaultValues => [ + { + label: i18n.t('meta.Sinks.DirtyData.Search.DirtyType'), + type: 'select', + name: 'dirtyType', + props: { + allowClear: true, + options: [ + { + label: 'DeserializeError', + value: 'DeserializeError', + }, + { + label: 'FieldMappingError', + value: 'FieldMappingError', + }, + { + label: 'LoadError', + value: 'LoadError', + }, + ], + }, + }, + { + label: i18n.t('meta.Sinks.DirtyData.Search.ServerType'), + type: 'select', + name: 'serverType', + props: { + allowClear: true, + options: [ + { + label: 'TubeMQ', + value: 'TubeMQ', + }, + { + label: 'Iceberg', + value: 'Iceberg', + }, + ], + }, + }, + { + label: i18n.t('meta.Sinks.DirtyTrend.DataTimeUnit'), + type: 'select', + name: 'dataTimeUnit', + initialValue: 'D', + props: { + options: [ + { + label: i18n.t('meta.Sinks.DirtyTrend.Day'), + value: 'D', + }, + { + label: i18n.t('meta.Sinks.DirtyTrend.Hour'), + value: 'H', + }, + ], + }, + }, + + { + type: 'datepicker', + label: i18n.t('pages.GroupDetail.Audit.StartDate'), + name: 'startTime', + props: values => { + return { + allowClear: true, + showTime: values.dataTimeUnit === 'H', + format: values.dataTimeUnit === 'D' ? 'YYYY-MM-DD' : 'YYYY-MM-DD HH', + }; + }, + initialValue: dayjs(trendOptions.startTime), + rules: [ + { required: true }, + ({ getFieldValue }) => ({ + validator(_, value) { + if (Boolean(value)) { + if (value.isAfter(dayjs())) { + return Promise.reject(new Error(i18n.t('meta.Sinks.DirtyData.StartTimeError'))); + } + } + return Promise.resolve(); + }, + }), + ], + }, + { + type: 'datepicker', + label: i18n.t('pages.GroupDetail.Audit.EndDate'), + name: 'endTime', + initialValue: dayjs(trendOptions.endTime), + props: values => { + return { + allowClear: true, + showTime: values.dataTimeUnit === 'H', + format: values.dataTimeUnit === 'D' ? 'YYYY-MM-DD' : 'YYYY-MM-DD HH', + }; + }, + rules: [ + { required: true }, + ({ getFieldValue }) => ({ + validator(_, value) { + if (Boolean(value)) { + if (value.isAfter(dayjs())) { + return Promise.reject( + new Error(i18n.t('meta.Sinks.DirtyData.endTimeNotGreaterThanStartTime')), + ); + } + const timeDiff = value.diff(getFieldValue('startTime'), 'day'); + if (timeDiff <= 7) { + return Promise.resolve(); + } + return Promise.reject(new Error(i18n.t('meta.Sinks.DirtyData.TimeIntervalError'))); + } + return Promise.resolve(); + }, + }), + ], + }, + { + type: ( + + ), + }, + ]; + + const onFilter = allValues => { + setOptions(prev => ({ + ...prev, + ...allValues, + startTime: allValues.startTime ? +allValues.startTime.$d : '', + endTime: allValues.endTime ? +allValues.endTime.$d : '', + })); + }; + + const onTrendFilter = allValues => { + setTrendOption(prev => ({ + ...prev, + ...allValues, + startTime: allValues.startTime + ? allValues.dataTimeUnit === 'H' + ? dayjs(allValues.startTime.$d).format('YYYYMMDDHH') + : dayjs(allValues.startTime.$d).format('YYYYMMDD') + : '', + endTime: allValues.endTime + ? allValues.dataTimeUnit === 'H' + ? dayjs(allValues.endTime.$d).format('YYYYMMDDHH') + : dayjs(allValues.endTime.$d).format('YYYYMMDD') + : '', + })); + }; + const scroll = { x: 2000 }; + const toChartData = trendData => { + return { + legend: { + data: trendData.map(item => item.reportTime), + }, + tooltip: { + trigger: 'axis', + }, + xAxis: { + type: 'category', + data: trendData.map(item => item.reportTime), + }, + yAxis: { + type: 'value', + }, + series: trendData.map(item => ({ + name: item.reportTime, + type: 'line', + data: trendData.map(item => item.count), + })), + }; + }; + const items: TabsProps['items'] = [ + { + key: 'detail', + label: i18n.t('meta.Sinks.DirtyData.Detail'), + children: ( + <> + + + + ), + }, + { + key: 'trend', + label: i18n.t('meta.Sinks.DirtyData.Trend'), + children: ( + <> + + + + ), + }, + ]; + const onTabChange = (key: string) => { + setTabValue(key); + }; + useEffect(() => { + onTabChange('detail'); + }, [modalProps.open]); + return ( + <> + {contextHolder} + { + onTabChange('detail'); + }} + > +
+ +
+
+ + ); +}; + +export default Comp; diff --git a/inlong-distribution/src/main/assemblies/sort-connectors-v1.18.xml b/inlong-distribution/src/main/assemblies/sort-connectors-v1.18.xml index 04e1fce16b2..0915b62483d 100644 --- a/inlong-distribution/src/main/assemblies/sort-connectors-v1.18.xml +++ b/inlong-distribution/src/main/assemblies/sort-connectors-v1.18.xml @@ -44,6 +44,14 @@ 0644 + + ../inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/target + inlong-sort/connectors + + sort-connector-kafka-v1.18-${project.version}.jar + + 0644 + diff --git a/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/entity/DirtyQueryLogEntity.java b/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/entity/DirtyQueryLogEntity.java new file mode 100644 index 00000000000..3554284f344 --- /dev/null +++ b/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/entity/DirtyQueryLogEntity.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.dao.entity; + +import lombok.Data; + +import java.io.Serializable; +import java.util.Date; + +@Data +public class DirtyQueryLogEntity implements Serializable { + + private static final long serialVersionUID = 1L; + + private Integer id; + private String md5; + private String requestParams; + private String taskId; + private Integer isDeleted; + private String creator; + private String modifier; + private Date createTime; + private Date modifyTime; + private Integer version; +} \ No newline at end of file diff --git a/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/DirtyQueryLogEntityMapper.java b/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/DirtyQueryLogEntityMapper.java new file mode 100644 index 00000000000..08db57441ac --- /dev/null +++ b/inlong-manager/manager-dao/src/main/java/org/apache/inlong/manager/dao/mapper/DirtyQueryLogEntityMapper.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.dao.mapper; + +import org.apache.inlong.manager.dao.entity.DirtyQueryLogEntity; + +import org.apache.ibatis.annotations.Param; +import org.springframework.stereotype.Repository; + +@Repository +public interface DirtyQueryLogEntityMapper { + + int updateByIdSelective(DirtyQueryLogEntity record); + + int insert(DirtyQueryLogEntity record); + + DirtyQueryLogEntity selectByPrimaryKey(Integer id); + + DirtyQueryLogEntity selectByMd5(String md5); + + void updateToTimeout(@Param("beforeMinutes") Integer beforeMinutes); + +} \ No newline at end of file diff --git a/inlong-manager/manager-dao/src/main/resources/mappers/DirtyQueryLogEntityMapper.xml b/inlong-manager/manager-dao/src/main/resources/mappers/DirtyQueryLogEntityMapper.xml new file mode 100644 index 00000000000..0fb1ce8e5f9 --- /dev/null +++ b/inlong-manager/manager-dao/src/main/resources/mappers/DirtyQueryLogEntityMapper.xml @@ -0,0 +1,93 @@ + + + + + + + + + + + + + + + + + + + + id, md5, request_params, task_id, is_deleted, creator, modifier, create_time, modify_time, version + + + + + insert into dirty_query_log (id, md5, request_params, + task_id, creator, modifier) + values (#{id,jdbcType=INTEGER}, #{md5,jdbcType=VARCHAR}, #{requestParams,jdbcType=VARCHAR}, + #{taskId,jdbcType=VARCHAR}, #{creator,jdbcType=VARCHAR}, #{modifier,jdbcType=VARCHAR}) + + + update dirty_query_log + + + md5 = #{md5,jdbcType=VARCHAR}, + + + request_params = #{requestParams,jdbcType=VARCHAR}, + + + task_id = #{taskId,jdbcType=VARCHAR}, + + + is_deleted = #{isDeleted,jdbcType=INTEGER}, + + + modifier = #{modifier,jdbcType=VARCHAR}, + + version = #{version,jdbcType=INTEGER} + 1 + + where id = #{id,jdbcType=INTEGER} + and version = #{version,jdbcType=INTEGER} + + + update dirty_query_log + + is_deleted = id + + + is_deleted = 0 + and create_time <= DATE_ADD(NOW(), INTERVAL -#{beforeMinutes, jdbcType=INTEGER} MINUTE) + + + \ No newline at end of file diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/group/InlongGroupInfo.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/group/InlongGroupInfo.java index 75fd9f35b93..1d6aa3b63c7 100644 --- a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/group/InlongGroupInfo.java +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/group/InlongGroupInfo.java @@ -142,6 +142,10 @@ public abstract class InlongGroupInfo extends BaseInlongGroup { @ApiModelProperty("Schedule type") private Integer scheduleType; + // schedule engine type, support [Quartz, Airflow, DolphinScheduler] + @ApiModelProperty("Schedule engine") + private String scheduleEngine; + // time unit for offline task schedule interval, support [month, week, day, hour, minute, oneround] // Y=year, M=month, W=week, D=day, H=hour, I=minute, O=oneround @ApiModelProperty("TimeUnit for schedule interval") diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/AirflowConnection.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/AirflowConnection.java new file mode 100644 index 00000000000..deb056ed4f7 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/AirflowConnection.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.airflow; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@NoArgsConstructor +@AllArgsConstructor +@JsonIgnoreProperties(ignoreUnknown = true) +@ApiModel(description = "Full representation of the connection.") +public class AirflowConnection { + + @JsonProperty("connection_id") + @ApiModelProperty("The connection ID.") + private String connectionId; + + @JsonProperty("conn_type") + @ApiModelProperty("The connection type.") + private String connType; + + @JsonProperty("description") + @ApiModelProperty("The description of the connection.") + private String description; + + @JsonProperty("host") + @ApiModelProperty("Host of the connection.") + private String host; + + @JsonProperty("login") + @ApiModelProperty("Login of the connection.") + private String login; + + @JsonProperty("schema") + @ApiModelProperty("Schema of the connection.") + private String schema; + + @JsonProperty("port") + @ApiModelProperty("Port of the connection.") + private Integer port; + + @JsonProperty("password") + @ApiModelProperty("Password of the connection.") + private String password; + + @JsonProperty("extra") + @ApiModelProperty("Additional information description of the connection.") + private String extra; +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAG.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAG.java new file mode 100644 index 00000000000..578eadb1514 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAG.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.airflow; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +@Data +@JsonIgnoreProperties(ignoreUnknown = true) +@ApiModel(description = "DAG Description Information.") +public class DAG { + + @JsonProperty("dag_id") + @ApiModelProperty("The ID of the DAG.") + private String dagId; + + @JsonProperty("root_dag_id") + @ApiModelProperty("If the DAG is SubDAG then it is the top level DAG identifier. Otherwise, null.") + private String rootDagId; + + @JsonProperty("is_paused") + @ApiModelProperty("Whether the DAG is paused.") + private Boolean isPaused; + + @JsonProperty("is_active") + @ApiModelProperty("Whether the DAG is currently seen by the scheduler(s).") + private Boolean isActive; + + @JsonProperty("description") + @ApiModelProperty("User-provided DAG description, which can consist of several sentences or paragraphs that describe DAG contents.") + private String description; +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAGCollection.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAGCollection.java new file mode 100644 index 00000000000..7a52548f417 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAGCollection.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.airflow; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +import java.util.List; + +@Data +@JsonIgnoreProperties(ignoreUnknown = true) +@ApiModel(description = "Collection of DAGs.") +public class DAGCollection { + + @JsonProperty("dags") + @ApiModelProperty("List of DAGs.") + private List dags = null; + + @JsonProperty("total_entries") + @ApiModelProperty("The length of DAG list.") + private Integer totalEntries; +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAGRun.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAGRun.java new file mode 100644 index 00000000000..e9384c75da4 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAGRun.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.airflow; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +@Data +@JsonIgnoreProperties(ignoreUnknown = true) +@ApiModel(description = "DAGRun Description Information.") +public class DAGRun { + + @JsonProperty("conf") + @ApiModelProperty("JSON object describing additional configuration parameters.") + private Object conf; + + @JsonProperty("dag_id") + @ApiModelProperty("Airflow DAG id.") + private String dagId; + + @JsonProperty("dag_run_id") + @ApiModelProperty("Airflow DAGRun id (Nullable).") + private String dagRunId; + + @JsonProperty("end_date") + @ApiModelProperty("The end time of this DAGRun.") + private String endDate; + + @JsonProperty("start_date") + @ApiModelProperty("The start time of this DAGRun.") + private String startDate; +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAGRunConf.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAGRunConf.java new file mode 100644 index 00000000000..4154c2526ce --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/DAGRunConf.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.airflow; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; +import lombok.NoArgsConstructor; + +@Data +@Builder +@NoArgsConstructor +@AllArgsConstructor +@JsonIgnoreProperties(ignoreUnknown = true) +@ApiModel(description = "DAGRunConf Description Information.") +public class DAGRunConf { + + @JsonProperty("inlong_group_id") + @ApiModelProperty("Specify the Inlong group ID") + private String inlongGroupId; + + @JsonProperty("start_time") + @ApiModelProperty("The start time of DAG scheduling.") + private long startTime; + + @JsonProperty("end_time") + @ApiModelProperty("The end time of DAG scheduling.") + private long endTime; + + @JsonProperty("boundary_type") + @ApiModelProperty("The offline task boundary type.") + private String boundaryType; + + @JsonProperty("cron_expr") + @ApiModelProperty("Cron expression.") + private String cronExpr; + + @JsonProperty("seconds_interval") + @ApiModelProperty("Time interval (in seconds).") + private String secondsInterval; + + @JsonProperty("connection_id") + @ApiModelProperty("Airflow Connection Id of Inlong Manager.") + private String connectionId; + + @JsonProperty("timezone") + @ApiModelProperty("The timezone.") + private String timezone; +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/Error.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/Error.java new file mode 100644 index 00000000000..3eb76fd6773 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/airflow/Error.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.airflow; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +import java.math.BigDecimal; + +@Data +@JsonIgnoreProperties(ignoreUnknown = true) +@ApiModel(description = "[RFC7807](https://tools.ietf.org/html/rfc7807) compliant response. ") +public class Error { + + @JsonProperty("detail") + @ApiModelProperty("Error Details.") + private String detail; + + @JsonProperty("instance") + @ApiModelProperty("Error of the instance.") + private String instance; + + @JsonProperty("status") + @ApiModelProperty("Error of the status.") + private BigDecimal status; + + @JsonProperty("title") + @ApiModelProperty("Error of the title.") + private String title; + + @JsonProperty("type") + @ApiModelProperty("Error of the type.") + private String type; +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DSTaskDefinition.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DSTaskDefinition.java new file mode 100644 index 00000000000..700638b63c9 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DSTaskDefinition.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.dolphinschedule; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +@Data +public class DSTaskDefinition { + + @ApiModelProperty("DolphinScheduler task definition code") + @JsonProperty("code") + private long code; + + @ApiModelProperty("DolphinScheduler task definition code") + @JsonProperty("delayTime") + private String delayTime; + + @ApiModelProperty("DolphinScheduler task definition description") + @JsonProperty("description") + private String description; + + @ApiModelProperty("DolphinScheduler task definition environment code") + @JsonProperty("environmentCode") + private int environmentCode; + + @ApiModelProperty("DolphinScheduler task fail retry interval") + @JsonProperty("failRetryInterval") + private String failRetryInterval; + + @ApiModelProperty("DolphinScheduler task definition fail retry times") + @JsonProperty("failRetryTimes") + private String failRetryTimes; + + @ApiModelProperty("DolphinScheduler task definition flag") + @JsonProperty("flag") + private String flag; + + @ApiModelProperty("DolphinScheduler task definition isCache") + @JsonProperty("isCache") + private String isCache; + + @ApiModelProperty("DolphinScheduler task definition name") + @JsonProperty("name") + private String name; + + @ApiModelProperty("DolphinScheduler task definition params") + @JsonProperty("taskParams") + private DSTaskParams taskParams; + + @ApiModelProperty("DolphinScheduler task definition priority") + @JsonProperty("taskPriority") + private String taskPriority; + + @ApiModelProperty("DolphinScheduler task definition type") + @JsonProperty("taskType") + private String taskType; + + @ApiModelProperty("DolphinScheduler task definition timeout") + @JsonProperty("timeout") + private int timeout; + + @ApiModelProperty("DolphinScheduler task definition timeout flag") + @JsonProperty("timeoutFlag") + private String timeoutFlag; + + @ApiModelProperty("DolphinScheduler task definition timeout notify strategy") + @JsonProperty("timeoutNotifyStrategy") + private String timeoutNotifyStrategy; + + @ApiModelProperty("DolphinScheduler task definition worker group") + @JsonProperty("workerGroup") + private String workerGroup; + + @ApiModelProperty("DolphinScheduler task definition apu quota") + @JsonProperty("cpuQuota") + private int cpuQuota; + + @ApiModelProperty("DolphinScheduler task definition memory max") + @JsonProperty("memoryMax") + private int memoryMax; + + @ApiModelProperty("DolphinScheduler task definition execute type") + @JsonProperty("taskExecuteType") + private String taskExecuteType; + + public DSTaskDefinition() { + this.delayTime = "0"; + this.description = ""; + this.environmentCode = -1; + this.failRetryInterval = "1"; + this.failRetryTimes = "0"; + this.flag = "YES"; + this.isCache = "NO"; + this.taskPriority = "MEDIUM"; + this.taskType = "SHELL"; + this.timeoutFlag = "CLOSE"; + this.timeoutNotifyStrategy = ""; + this.workerGroup = "default"; + this.cpuQuota = -1; + this.memoryMax = -1; + this.taskExecuteType = "BATCH"; + } +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DSTaskParams.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DSTaskParams.java new file mode 100644 index 00000000000..a5344f5facf --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DSTaskParams.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.dolphinschedule; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +import java.util.ArrayList; +import java.util.List; +@Data +public class DSTaskParams { + + @ApiModelProperty("DolphinScheduler task params local params") + @JsonProperty("localParams") + private List localParams; + + @ApiModelProperty("DolphinScheduler task params raw script") + @JsonProperty("rawScript") + private String rawScript; + + @ApiModelProperty("DolphinScheduler task params resource list") + @JsonProperty("resourceList") + private List resourceList; + + public DSTaskParams() { + this.localParams = new ArrayList<>(); + this.resourceList = new ArrayList<>(); + this.rawScript = ""; + } +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DSTaskRelation.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DSTaskRelation.java new file mode 100644 index 00000000000..e853317df77 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DSTaskRelation.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.dolphinschedule; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +@Data +public class DSTaskRelation { + + @ApiModelProperty("DolphinScheduler task relation name") + @JsonProperty("name") + private String name; + + @ApiModelProperty("DolphinScheduler task relation pre-task code") + @JsonProperty("preTaskCode") + private int preTaskCode; + + @ApiModelProperty("DolphinScheduler task relation pre-task version") + @JsonProperty("preTaskVersion") + private int preTaskVersion; + + @ApiModelProperty("DolphinScheduler task relation post-task code") + @JsonProperty("postTaskCode") + private long postTaskCode; + + @ApiModelProperty("DolphinScheduler task relation post-task version") + @JsonProperty("postTaskVersion") + private int postTaskVersion; + + @ApiModelProperty("DolphinScheduler task relation condition type") + @JsonProperty("conditionType") + private String conditionType; + + @ApiModelProperty("DolphinScheduler task relation condition params") + @JsonProperty("conditionParams") + private Object conditionParams; + + public DSTaskRelation() { + this.name = ""; + this.conditionType = "NONE"; + } +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DScheduleInfo.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DScheduleInfo.java new file mode 100644 index 00000000000..ac45b26f197 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/schedule/dolphinschedule/DScheduleInfo.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.schedule.dolphinschedule; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +@Data +public class DScheduleInfo { + + @ApiModelProperty("DolphinScheduler schedule start time") + @JsonProperty("startTime") + private String startTime; + + @ApiModelProperty("DolphinScheduler schedule end time") + @JsonProperty("endTime") + private String endTime; + + @ApiModelProperty("DolphinScheduler schedule crontab expression") + @JsonProperty("crontab") + private String crontab; + + @ApiModelProperty("DolphinScheduler schedule timezone id") + @JsonProperty("timezoneId") + private String timezoneId; + +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/BaseStreamSink.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/BaseStreamSink.java index d8df7f4a285..6ad6c9b5230 100644 --- a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/BaseStreamSink.java +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/BaseStreamSink.java @@ -17,12 +17,18 @@ package org.apache.inlong.manager.pojo.sink; +import org.apache.inlong.manager.common.enums.ErrorCodeEnum; +import org.apache.inlong.manager.common.exceptions.BusinessException; +import org.apache.inlong.manager.common.util.JsonUtils; + import io.swagger.annotations.ApiModel; import io.swagger.annotations.ApiModelProperty; import lombok.AllArgsConstructor; import lombok.Data; import lombok.NoArgsConstructor; +import javax.validation.constraints.NotNull; + /** * The base parameter class of StreamSink, support user extend their own business params. */ @@ -32,6 +38,9 @@ @ApiModel("Base info of stream sink") public class BaseStreamSink { + @ApiModelProperty("Enable data archiving") + private Boolean enableDataArchiving; + @ApiModelProperty("Transform sql") private String transformSql; @@ -40,4 +49,12 @@ public class BaseStreamSink { @ApiModelProperty("Stop consume time, yyyy-MM-dd HH:mm:ss format") private String stopConsumeTime; + + public static BaseStreamSink getFromJson(@NotNull String extParams) { + try { + return JsonUtils.parseObject(extParams, BaseStreamSink.class); + } catch (Exception e) { + throw new BusinessException(ErrorCodeEnum.SINK_INFO_INCORRECT.getMessage() + ": " + e.getMessage()); + } + } } diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataDetailResponse.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataDetailResponse.java new file mode 100644 index 00000000000..5ef774ee252 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataDetailResponse.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.sink; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +/** + * Dirty data detail info. + */ +@Data +@ApiModel("Dirty data detail info") +public class DirtyDataDetailResponse { + + @ApiModelProperty(value = "Dirty data partition") + private String dirtyDataPartition; + + @ApiModelProperty(value = "Data flow id") + private String dataFlowId; + + @ApiModelProperty(value = "Inlong group id") + private String groupId; + + @ApiModelProperty(value = "Inlong stream id") + private String streamId; + + @ApiModelProperty(value = "Report time") + private String reportTime; + + @ApiModelProperty(value = "Data time") + private String dataTime; + + @ApiModelProperty(value = "Server type") + private String serverType; + + @ApiModelProperty(value = "Dirty type") + private String dirtyType; + + @ApiModelProperty(value = "Dirty message") + private String dirtyMessage; + + @ApiModelProperty(value = "Ext info") + private String extInfo; + + @ApiModelProperty(value = "Dirty data") + private String dirtyData; +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataRequest.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataRequest.java new file mode 100644 index 00000000000..8b3fc32a6f8 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataRequest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.sink; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; +import lombok.EqualsAndHashCode; + +import java.util.List; + +/** + * Query request for Dirty data + */ +@Data +@EqualsAndHashCode(callSuper = false) +@ApiModel("Query request for Dirty data") +public class DirtyDataRequest { + + @ApiModelProperty(value = "Sink id list") + private List sinkIdList; + + @ApiModelProperty(value = "Key word") + private String keyword; + + @ApiModelProperty(value = "Server type") + private String serverType; + + @ApiModelProperty(value = "Dirty type") + private String dirtyType; + + @ApiModelProperty(value = "Start time") + private String startTime; + + @ApiModelProperty(value = "End time") + private String endTime; + + @ApiModelProperty(value = "Data count") + private Integer dataCount = 10; +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataResponse.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataResponse.java new file mode 100644 index 00000000000..08203db09a6 --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataResponse.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.sink; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +/** + * Dirty data info. + */ +@Data +@ApiModel("Dirty data info") +public class DirtyDataResponse { + + @ApiModelProperty(value = "Task id") + private String taskId; + +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataTrendDetailResponse.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataTrendDetailResponse.java new file mode 100644 index 00000000000..cc56cd93eee --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataTrendDetailResponse.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.sink; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; + +/** + * Dirty data detail info. + */ +@Data +@ApiModel("Dirty data trend detail info") +public class DirtyDataTrendDetailResponse { + + @ApiModelProperty(value = "Report time") + private String reportTime; + + @ApiModelProperty(value = "Data count") + private String count; + +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataTrendRequest.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataTrendRequest.java new file mode 100644 index 00000000000..b0e52420a2a --- /dev/null +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/DirtyDataTrendRequest.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.pojo.sink; + +import io.swagger.annotations.ApiModel; +import io.swagger.annotations.ApiModelProperty; +import lombok.Data; +import lombok.EqualsAndHashCode; + +import java.util.List; + +/** + * Query request for Dirty data + */ +@Data +@EqualsAndHashCode(callSuper = false) +@ApiModel("Query request for Dirty data") +public class DirtyDataTrendRequest { + + @ApiModelProperty(value = "Sink id list") + private List sinkIdList; + + @ApiModelProperty(value = "Data time unit") + private String dataTimeUnit; + + @ApiModelProperty(value = "Server type") + private String serverType; + + @ApiModelProperty(value = "Dirty type") + private String dirtyType; + + @ApiModelProperty(value = "Start time") + private String startTime; + + @ApiModelProperty(value = "End time") + private String endTime; +} diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/SinkRequest.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/SinkRequest.java index 24c544b943f..782817a50d7 100644 --- a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/SinkRequest.java +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/SinkRequest.java @@ -100,6 +100,9 @@ public abstract class SinkRequest { @Range(min = 0, max = 1, message = "default is 1, only supports [0: disable, 1: enable]") private Integer enableCreateResource = 1; + @ApiModelProperty("Enable data archiving") + private Boolean enableDataArchiving; + @ApiModelProperty(value = "Whether to start the process after saving or updating. Default is false") private Boolean startProcess = false; diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/StreamSink.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/StreamSink.java index 85fd72a1a44..79a5af38764 100644 --- a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/StreamSink.java +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/sink/StreamSink.java @@ -90,6 +90,9 @@ public abstract class StreamSink extends StreamNode { @ApiModelProperty(value = "Whether to enable create sink resource? 0: disable, 1: enable. Default is 1", notes = "Such as enable or disable to create Hive table") private Integer enableCreateResource = 1; + @ApiModelProperty("Enable data archiving") + private Boolean enableDataArchiving; + @ApiModelProperty("Backend operation log") private String operateLog; diff --git a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/util/MySQLSensitiveUrlUtils.java b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/util/MySQLSensitiveUrlUtils.java index fd804603019..32655da1fb2 100644 --- a/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/util/MySQLSensitiveUrlUtils.java +++ b/inlong-manager/manager-pojo/src/main/java/org/apache/inlong/manager/pojo/util/MySQLSensitiveUrlUtils.java @@ -71,11 +71,13 @@ public static String filterSensitive(String url) { } resultUrl = resultUrl.replaceAll(InlongConstants.REGEX_WHITESPACE, InlongConstants.EMPTY); - for (String key : SENSITIVE_REPLACE_PARAM_MAP.keySet()) { - resultUrl = StringUtils.replaceIgnoreCase(resultUrl, key + InlongConstants.EQUAL + "true", + String sensitiveKey = containSensitiveKey(resultUrl); + while (StringUtils.isNotBlank(sensitiveKey)) { + resultUrl = StringUtils.replaceIgnoreCase(resultUrl, sensitiveKey + InlongConstants.EQUAL + "true", InlongConstants.EMPTY); - resultUrl = StringUtils.replaceIgnoreCase(resultUrl, key + InlongConstants.EQUAL + "yes", + resultUrl = StringUtils.replaceIgnoreCase(resultUrl, sensitiveKey + InlongConstants.EQUAL + "yes", InlongConstants.EMPTY); + sensitiveKey = containSensitiveKey(resultUrl); } if (resultUrl.contains(InlongConstants.QUESTION_MARK)) { StringBuilder builder = new StringBuilder(); @@ -114,4 +116,14 @@ public static String filterSensitive(String url) { url, e.getMessage())); } } + + public static String containSensitiveKey(String url) { + for (String key : SENSITIVE_REPLACE_PARAM_MAP.keySet()) { + if (url.contains(key + InlongConstants.EQUAL + "true") + || url.contains(key + InlongConstants.EQUAL + "yes")) { + return key; + } + } + return null; + } } diff --git a/inlong-manager/manager-pojo/src/test/java/org/apache/inlong/manager/pojo/sink/mysql/MySQLSinkDTOTest.java b/inlong-manager/manager-pojo/src/test/java/org/apache/inlong/manager/pojo/sink/mysql/MySQLSinkDTOTest.java index 3955a1538cc..b4805af545b 100644 --- a/inlong-manager/manager-pojo/src/test/java/org/apache/inlong/manager/pojo/sink/mysql/MySQLSinkDTOTest.java +++ b/inlong-manager/manager-pojo/src/test/java/org/apache/inlong/manager/pojo/sink/mysql/MySQLSinkDTOTest.java @@ -36,6 +36,10 @@ public void testFilterSensitive() throws Exception { "jdbc:mysql://127.0.0.1:3306?autoReconnect=true&autoDeserialize=false&allowUrlInLocalInfile=false&allowLoadLocalInfile=false", originUrl); + originUrl = MySQLSinkDTO.filterSensitive( + "jdbc:mysql://address=(host=127.0.0.1)(port=3306)(allowLoadallowLoadLocalInfile=trueLocalInfile=true)"); + Assertions.assertEquals("jdbc:mysql://address=(host=127.0.0.1)(port=3306)()", originUrl); + originUrl = MySQLSinkDTO.filterSensitive( "jdbc:mysql://127.0.0.1:3306?autoReconnect=true&autoDeserialize = TRue&allowLoadLocalInfile=TRue&allowUrlInLocalInfile=TRue&allowLoadLocalInfileInPath=/"); Assertions.assertEquals( diff --git a/inlong-manager/manager-schedule/pom.xml b/inlong-manager/manager-schedule/pom.xml index a9d9fb3e1ed..82632a1aff8 100644 --- a/inlong-manager/manager-schedule/pom.xml +++ b/inlong-manager/manager-schedule/pom.xml @@ -73,5 +73,35 @@ junit-jupiter test + + org.testcontainers + testcontainers + ${testcontainers.version} + test + + + org.springframework.boot + spring-boot-test + ${spring.boot.version} + test + + + org.springframework.boot + spring-boot-starter-test + + + com.vaadin.external.google + android-json + + + com.jayway.jsonpath + json-path + + + org.junit.jupiter + junit-jupiter-api + + + diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/ScheduleEngineType.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/ScheduleEngineType.java index 71949ef7445..8ae586609dd 100644 --- a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/ScheduleEngineType.java +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/ScheduleEngineType.java @@ -23,7 +23,9 @@ public enum ScheduleEngineType { NONE("None"), - QUARTZ("Quartz"); + QUARTZ("Quartz"), + AIRFLOW("Airflow"), + DOLPHINSCHEDULER("DolphinScheduler"); private final String type; diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirFlowAPIConstant.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirFlowAPIConstant.java new file mode 100644 index 00000000000..e328d8fd0a0 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirFlowAPIConstant.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow; + +/** + * Contains constants for interacting with the Airflow API. + */ +public class AirFlowAPIConstant { + + public static final String DEFAULT_TIMEZONE = "Asia/Shanghai"; + public static final String INLONG_OFFLINE_DAG_TASK_PREFIX = "inlong_offline_task_"; + public static final String SUBMIT_OFFLINE_JOB_URI = "/inlong/manager/api/group/submitOfflineJob"; + + // AirflowConnection + public static final String LIST_CONNECTIONS_URI = "/api/v1/connections"; + public static final String GET_CONNECTION_URI = "/api/v1/connections/{connection_id}"; + + // DAG + public static final String LIST_DAGS_URI = "/api/v1/dags"; + public static final String UPDATE_DAG_URI = "/api/v1/dags/{dag_id}"; + + // DAGRun + public static final String TRIGGER_NEW_DAG_RUN_URI = "/api/v1/dags/{dag_id}/dagRuns"; + +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirflowScheduleClient.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirflowScheduleClient.java new file mode 100644 index 00000000000..bbb8e59149a --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirflowScheduleClient.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow; + +import org.apache.inlong.manager.pojo.schedule.ScheduleInfo; +import org.apache.inlong.manager.schedule.ScheduleEngineClient; +import org.apache.inlong.manager.schedule.ScheduleEngineType; + +import org.springframework.stereotype.Service; + +import javax.annotation.Resource; + +/** + * Built-in implementation of schedule engine client corresponding with {@link AirflowScheduleEngine}. + * AirflowScheduleClient simply invokes the {@link AirflowScheduleEngine} to register/unregister/update + * schedule info instead of calling a remote schedule service. + * */ +@Service +public class AirflowScheduleClient implements ScheduleEngineClient { + + @Resource + public AirflowScheduleEngine scheduleEngine; + + @Override + public boolean accept(String engineType) { + return ScheduleEngineType.AIRFLOW.getType().equalsIgnoreCase(engineType); + } + + @Override + public boolean register(ScheduleInfo scheduleInfo) { + return scheduleEngine.handleRegister(scheduleInfo); + } + + @Override + public boolean unregister(String groupId) { + return scheduleEngine.handleUnregister(groupId); + } + + @Override + public boolean update(ScheduleInfo scheduleInfo) { + return scheduleEngine.handleUpdate(scheduleInfo); + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirflowScheduleEngine.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirflowScheduleEngine.java new file mode 100644 index 00000000000..80d67f22818 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirflowScheduleEngine.java @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow; + +import org.apache.inlong.common.bounded.BoundaryType; +import org.apache.inlong.manager.pojo.schedule.ScheduleInfo; +import org.apache.inlong.manager.pojo.schedule.airflow.AirflowConnection; +import org.apache.inlong.manager.pojo.schedule.airflow.DAG; +import org.apache.inlong.manager.pojo.schedule.airflow.DAGCollection; +import org.apache.inlong.manager.pojo.schedule.airflow.DAGRun; +import org.apache.inlong.manager.pojo.schedule.airflow.DAGRunConf; +import org.apache.inlong.manager.schedule.ScheduleEngine; +import org.apache.inlong.manager.schedule.ScheduleUnit; +import org.apache.inlong.manager.schedule.airflow.api.AirflowResponse; +import org.apache.inlong.manager.schedule.airflow.api.connection.AirflowConnectionCreator; +import org.apache.inlong.manager.schedule.airflow.api.connection.AirflowConnectionGetter; +import org.apache.inlong.manager.schedule.airflow.api.dag.DAGCollectionUpdater; +import org.apache.inlong.manager.schedule.airflow.api.dag.DAGDeletor; +import org.apache.inlong.manager.schedule.airflow.api.dag.DAGUpdater; +import org.apache.inlong.manager.schedule.airflow.api.dagruns.DAGRunsTrigger; +import org.apache.inlong.manager.schedule.airflow.config.AirflowConfig; +import org.apache.inlong.manager.schedule.airflow.util.DAGUtil; +import org.apache.inlong.manager.schedule.airflow.util.DateUtil; +import org.apache.inlong.manager.schedule.exception.AirflowScheduleException; + +import com.google.common.collect.ImmutableMap; +import org.apache.mina.util.ConcurrentHashSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Service; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +import static org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant.DEFAULT_TIMEZONE; +import static org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant.INLONG_OFFLINE_DAG_TASK_PREFIX; +import static org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant.SUBMIT_OFFLINE_JOB_URI; +import static org.apache.inlong.manager.schedule.exception.AirflowScheduleException.AirflowErrorCode.DAG_DUPLICATE; +import static org.apache.inlong.manager.schedule.exception.AirflowScheduleException.AirflowErrorCode.INIT_CONNECTION_FAILED; +import static org.apache.inlong.manager.schedule.exception.AirflowScheduleException.AirflowErrorCode.SCHEDULE_ENGINE_SHUTDOWN_FAILED; +import static org.apache.inlong.manager.schedule.exception.AirflowScheduleException.AirflowErrorCode.SCHEDULE_TASK_REGISTER_FAILED; +import static org.apache.inlong.manager.schedule.exception.AirflowScheduleException.AirflowErrorCode.SCHEDULE_TASK_UPDATE_FAILED; +import static org.apache.inlong.manager.schedule.exception.AirflowScheduleException.AirflowErrorCode.TASK_DAG_SWITCH_FAILED; + +/** + * Response for processing the start/register/unregister/update/stop requests from {@link AirflowScheduleClient} + */ +@Service +public class AirflowScheduleEngine implements ScheduleEngine { + + private static final Logger LOGGER = LoggerFactory.getLogger(AirflowScheduleEngine.class); + private final Set scheduledJobSet = new ConcurrentHashSet<>(); + private AirflowServerClient serverClient; + private AirflowConfig airflowConfig; + + public AirflowScheduleEngine(AirflowServerClient serverClient, AirflowConfig airflowConfig) { + this.serverClient = serverClient; + this.airflowConfig = airflowConfig; + start(); + } + + @Override + public void start() { + try { + // Create authentication information for the Inlong Manger API used by AirFlow + initConnection(); + // Check if DagCleaner and DagCreator exist and unpause them + switchOriginalDAG(false); + // Start all task DAGs and load all DAG ID(Group Id) into the local cache + switchAllTaskDAG(false); + LOGGER.info("Airflow initialization succeeded."); + } catch (Exception e) { + LOGGER.error("Airflow initialization failed.", e); + } + } + + private void initConnection() throws Exception { + LOGGER.info("Initializing Inlong Manager AirflowConnection for Airflow ... "); + // Check if Airflow has the Inlong AirflowConnection + AirflowResponse response = serverClient.sendRequest( + new AirflowConnectionGetter(airflowConfig.getConnectionId())); + if (!response.isSuccess()) { + AirflowConnection newConn = new AirflowConnection(airflowConfig.getConnectionId(), "HTTP", "", + airflowConfig.getInlongManagerHost(), airflowConfig.getInlongUsername(), SUBMIT_OFFLINE_JOB_URI, + airflowConfig.getInlongManagerPort(), airflowConfig.getInlongPassword(), ""); + response = serverClient.sendRequest(new AirflowConnectionCreator(newConn)); + LOGGER.info("AirflowConnection registration response: {}", response.toString()); + if (!response.isSuccess()) { + LOGGER.error("Initialization connection failed."); + throw new AirflowScheduleException(INIT_CONNECTION_FAILED, "Initialization connection failed."); + } + } + } + + private void switchOriginalDAG(boolean isPaused) { + for (String dagId : Arrays.asList(airflowConfig.getDagCleanerId(), airflowConfig.getDagCreatorId())) { + try { + AirflowResponse response = serverClient.sendRequest(new DAGUpdater(dagId, isPaused)); + LOGGER.info("Response to {} the original DAG : {}", isPaused ? "stop" : "start", response.toString()); + if (!response.isSuccess()) { + throw new AirflowScheduleException(TASK_DAG_SWITCH_FAILED, + String.format("%s does not exist or failed to %s.", dagId, (isPaused ? "stop" : "start"))); + } + } catch (Exception e) { + LOGGER.error("The original DAG {} failed.", isPaused ? "stop" : "start", e); + throw new AirflowScheduleException(TASK_DAG_SWITCH_FAILED, + String.format("The original DAG %s failed: %s.", isPaused ? "stop" : "start", e.getMessage())); + } + } + } + + private void switchAllTaskDAG(boolean isPaused) { + try { + AirflowResponse response = serverClient.sendRequest( + new DAGCollectionUpdater(INLONG_OFFLINE_DAG_TASK_PREFIX, isPaused)); + LOGGER.info("Response to {} task DAG : {}", isPaused ? "stop" : "start", response.toString()); + if (!response.isSuccess()) { + throw new AirflowScheduleException(TASK_DAG_SWITCH_FAILED, + String.format("Failed to %s task DAGs.", isPaused ? "stop" : "start")); + } + if (!isPaused) { + List dagList = response.getData().getDags(); + if (dagList != null) { + dagList.forEach(dag -> scheduledJobSet + .add(dag.getDagId().substring(INLONG_OFFLINE_DAG_TASK_PREFIX.length() - 1))); + } + } + } catch (Exception e) { + LOGGER.error("Failed to {} task DAGs.", isPaused ? "stop" : "start", e); + throw new AirflowScheduleException(TASK_DAG_SWITCH_FAILED, + String.format("Failed to %s task DAGs: %s", isPaused ? "stop" : "start", e.getMessage())); + } + } + + @Override + public boolean handleRegister(ScheduleInfo scheduleInfo) { + try { + LOGGER.info("Registering DAG for {}", scheduleInfo.getInlongGroupId()); + return doRegister(scheduleInfo, true); + } catch (Exception e) { + LOGGER.error("The Airflow scheduling task with Group ID {} failed to register.", + scheduleInfo.getInlongGroupId(), e); + throw new AirflowScheduleException(SCHEDULE_TASK_REGISTER_FAILED, + String.format("The Airflow scheduling task with Group ID %s failed to register: %s", + scheduleInfo.getInlongGroupId(), e.getMessage())); + } + } + + @Override + public boolean handleUnregister(String groupId) { + LOGGER.info("Unregistering Airflow Dag with GroupId {} ", groupId); + if (scheduledJobSet.contains(groupId)) { + try { + if (!completelyDelete(DAGUtil.buildDAGIdByGroupId(groupId))) { + return false; + } + } catch (Exception e) { + LOGGER.warn("May not be completely removed {}", groupId, e); + } + } + scheduledJobSet.remove(groupId); + LOGGER.info("Un-registered airflow schedule info for {}", groupId); + return true; + } + + private boolean completelyDelete(String groupId) throws Exception { + // Trigger the removal of the DAG file for the Cleaner DAG + DAGRunConf dagRunConf = DAGRunConf.builder() + .inlongGroupId(DAGUtil.buildDAGIdByGroupId(groupId)).build(); + AirflowResponse response = serverClient.sendRequest( + new DAGRunsTrigger(airflowConfig.getDagCleanerId(), ImmutableMap.of("conf", dagRunConf))); + LOGGER.info("Response to DAG file clearing: {}", response.toString()); + if (!response.isSuccess()) { + LOGGER.warn("Failed to delete DAG file corresponding to {}.", groupId); + return false; + } + // Delete DAG tasks that have been loaded into memory + AirflowResponse deleteResponse = serverClient.sendRequest(new DAGDeletor(groupId)); + LOGGER.info("Response to DAG scheduling instance clearing: {}", deleteResponse.toString()); + if (!deleteResponse.isSuccess()) { + LOGGER.warn("Failed to delete DAG instance corresponding to {}.", groupId); + } + return deleteResponse.isSuccess(); + } + + @Override + public boolean handleUpdate(ScheduleInfo scheduleInfo) { + try { + LOGGER.info("Updating DAG for {}", scheduleInfo.getInlongGroupId()); + return doRegister(scheduleInfo, false); + } catch (Exception e) { + LOGGER.error("The Airflow scheduling task with Group ID {} failed to update.", + scheduleInfo.getInlongGroupId(), e); + throw new AirflowScheduleException(SCHEDULE_TASK_UPDATE_FAILED, + String.format("The Airflow scheduling task with Group ID %s failed to update: %s", + scheduleInfo.getInlongGroupId(), e.getMessage())); + } + } + + public boolean doRegister(ScheduleInfo scheduleInfo, boolean isFirst) throws Exception { + if (isFirst && scheduledJobSet.contains(scheduleInfo.getInlongGroupId())) { + throw new AirflowScheduleException(DAG_DUPLICATE, + String.format("Group %s is already registered", scheduleInfo.getInlongGroupId())); + } + DAGRunConf.DAGRunConfBuilder confBuilder = DAGRunConf.builder() + .inlongGroupId(scheduleInfo.getInlongGroupId()) + .startTime(scheduleInfo.getStartTime().getTime()) + .endTime(scheduleInfo.getEndTime().getTime()) + .boundaryType(BoundaryType.TIME.getType()) + .connectionId(airflowConfig.getConnectionId()) + .timezone(DEFAULT_TIMEZONE); + if (scheduleInfo.getScheduleType() == 1) { + confBuilder = confBuilder.cronExpr(scheduleInfo.getCrontabExpression()); + } else { + confBuilder = confBuilder.secondsInterval(DateUtil.intervalToSeconds(scheduleInfo.getScheduleInterval(), + scheduleInfo.getScheduleUnit())) + .startTime(ScheduleUnit.getScheduleUnit(scheduleInfo.getScheduleUnit()) == ScheduleUnit.ONE_ROUND + ? scheduleInfo.getEndTime().getTime() + : scheduleInfo.getStartTime().getTime()); + } + DAGRunConf dagRunConf = confBuilder.build(); + AirflowResponse response = serverClient.sendRequest( + new DAGRunsTrigger(airflowConfig.getDagCreatorId(), ImmutableMap.of("conf", dagRunConf))); + LOGGER.info("DAG {} response: {}", isFirst ? "registration" : "update", response.toString()); + if (response.isSuccess()) { + scheduledJobSet.add(scheduleInfo.getInlongGroupId()); + } + return response.isSuccess(); + } + + @Override + public void stop() { + try { + switchOriginalDAG(true); + switchAllTaskDAG(true); + } catch (Exception e) { + LOGGER.error("Airflow Schedule Engine shutdown failed: ", e); + throw new AirflowScheduleException(SCHEDULE_ENGINE_SHUTDOWN_FAILED, + String.format("Airflow Schedule Engine shutdown failed: %s", e.getMessage())); + } + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirflowServerClient.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirflowServerClient.java new file mode 100644 index 00000000000..be67a364758 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/AirflowServerClient.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow; + +import org.apache.inlong.manager.pojo.schedule.airflow.Error; +import org.apache.inlong.manager.schedule.airflow.api.AirflowApi; +import org.apache.inlong.manager.schedule.airflow.api.AirflowResponse; +import org.apache.inlong.manager.schedule.airflow.config.AirflowConfig; + +import com.fasterxml.jackson.databind.ObjectMapper; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.Response; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * A unified class used for Airflow RESTful API processing. + */ +public class AirflowServerClient { + + private static final Logger logger = LoggerFactory.getLogger(AirflowServerClient.class); + private final OkHttpClient httpClient; + private final AirflowConfig config; + private final ObjectMapper objectMapper; + + public AirflowServerClient(OkHttpClient httpClient, AirflowConfig config) { + this.httpClient = httpClient; + this.config = config; + this.objectMapper = new ObjectMapper(); + } + + /** + * Send request and parse response + * + * @param apiEndpoint apiEndpoint + * @param Response to Generic Types + * @return Parsed response object + * @throws IOException Network request exception + */ + public AirflowResponse sendRequest(AirflowApi apiEndpoint) throws IOException { + Request request = apiEndpoint.buildRequest(config.getBaseUrl()); + try (Response response = httpClient.newCall(request).execute()) { + String responseBody = response.body().string(); + if (response.isSuccessful()) { + return new AirflowResponse<>(true, objectMapper.readValue(responseBody, apiEndpoint.getResponseType())); + } else { + logger.error("Airflow Web API Request failed, status code: {} , detail: {}", + response.code(), objectMapper.readValue(responseBody, Error.class).getDetail()); + return new AirflowResponse<>(false, null); + } + } + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/AirflowApi.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/AirflowApi.java new file mode 100644 index 00000000000..4ff1a3284df --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/AirflowApi.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.api; + +import okhttp3.Request; +import okhttp3.RequestBody; +import org.springframework.http.HttpMethod; + +import java.util.Map; +/** + * Represents a generic interface for defining and constructing API requests to interact with Airflow. + * This interface provides methods for specifying HTTP methods, endpoint paths, parameters, + * request bodies, and constructing complete requests. + * @param the type of the response expected from the API, allowing flexibility for various response types. + */ +public interface AirflowApi { + + /** + * Get HTTP Method + * @return HTTP Method + */ + HttpMethod getMethod(); + + /** + * Get the requested path (relative to baseUrl) + * @return Request path + */ + String getPath(); + + /** + * Get path parameters to replace placeholders in the path (e.g. : "/api/v1/dags/{dag_id}/dagRuns") + * @return Path parameter map + */ + Map getPathParams(); + + /** + * Get query parameters (e.g. "?Key=value") + * @return GET parameter map + */ + Map getQueryParams(); + + /** + * Get the request body (applicable to methods such as POST, PUT, etc.) + * @return Post RequestBody Object + */ + RequestBody getRequestBody(); + + /** + * Constructing a complete Request object + * @param baseUrl Base URL + * @return Constructed Request object + */ + Request buildRequest(String baseUrl); + + /** + * Returns the type of the response expected from this method. + * @return The expected response type. + */ + Class getResponseType(); +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/AirflowResponse.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/AirflowResponse.java new file mode 100644 index 00000000000..60e0ef63668 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/AirflowResponse.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.api; + +/** + * A generic response wrapper for handling responses from Airflow services. + * @param the type of data included in the response, allowing flexibility for various data types. + */ +public class AirflowResponse { + + private final boolean success; + private final T data; + + public AirflowResponse(boolean success, T data) { + this.success = success; + this.data = data; + } + + public boolean isSuccess() { + return success; + } + + public T getData() { + return data; + } + + @Override + public String toString() { + return "AirflowResponse{" + + "success=" + success + + ", data=" + data + + '}'; + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/BaseAirflowApi.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/BaseAirflowApi.java new file mode 100644 index 00000000000..18a1ed5206a --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/BaseAirflowApi.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.api; + +import org.apache.inlong.manager.schedule.exception.AirflowScheduleException; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; +import lombok.extern.slf4j.Slf4j; +import okhttp3.MediaType; +import okhttp3.Request; +import okhttp3.RequestBody; +import org.springframework.http.HttpMethod; + +import java.util.List; +import java.util.Map; + +import static org.apache.inlong.manager.schedule.exception.AirflowScheduleException.AirflowErrorCode.BUILD_REQUEST_BODY_FAILED; + +/** + * The basic implementation of Airflow API interface. + * + * @param the type of the response expected from the API, allowing flexibility for various response types. + */ + +@Slf4j +public abstract class BaseAirflowApi implements AirflowApi { + + protected static final ObjectMapper objectMapper = new ObjectMapper(); + protected Map pathParams = Maps.newHashMap(); + protected Map queryParams = Maps.newHashMap(); + protected Map requestBodyParams = Maps.newHashMap(); + + @Override + public abstract HttpMethod getMethod(); + + @Override + public abstract String getPath(); + + @Override + public abstract Class getResponseType(); + + @Override + public Map getPathParams() { + return pathParams; + } + + @Override + public Map getQueryParams() { + return queryParams; + } + + /** + * Create JSON request body + * @return RequestBody Object + */ + @Override + public RequestBody getRequestBody() { + try { + return RequestBody.create(MediaType.parse("application/json; charset=utf-8"), + objectMapper.writeValueAsString(requestBodyParams)); + } catch (Exception e) { + log.error("Airflow request body construction failed: {}", e.getMessage(), e); + throw new AirflowScheduleException(BUILD_REQUEST_BODY_FAILED, + String.format("Airflow request body construction failed: %s", e.getMessage())); + } + } + + @Override + public Request buildRequest(String baseUrl) { + // Build a complete URL + String path = buildPathParams(getPath(), getPathParams()); + String url = baseUrl + path; + + // Add query parameters + if (!getQueryParams().isEmpty()) { + String queryString = buildQueryString(getQueryParams()); + url += "?" + queryString; + } + + // Build Request Builder + Request.Builder builder = new Request.Builder().url(url); + + // Set requests based on HTTP methods + switch (getMethod()) { + case GET: + builder.get(); + break; + case POST: + builder.post(getRequestBody()); + break; + case PATCH: + builder.patch(getRequestBody()); + break; + case PUT: + builder.put(getRequestBody()); + break; + case DELETE: + if (!requestBodyParams.isEmpty()) { + builder.delete(getRequestBody()); + } else { + builder.delete(); + } + break; + default: + throw new IllegalArgumentException("Unsupported HTTP method: " + getMethod()); + } + return builder.build(); + } + + private String buildPathParams(String path, Map pathParams) { + for (Map.Entry entry : pathParams.entrySet()) { + path = path.replace("{" + entry.getKey() + "}", entry.getValue()); + } + return path; + } + + private String buildQueryString(Map queryParams) { + StringBuilder sb = new StringBuilder(); + // Multiple values can be specified for the same parameter name in the Get parameter. + // (e.g. "?Key=value1&Key=value2") + queryParams.forEach((key, value) -> { + if (value instanceof List) { + ((List) value).forEach(item -> sb.append(key).append("=").append(item).append("&")); + } else { + sb.append(key).append("=").append(value).append("&"); + } + }); + if (sb.length() > 0) { + sb.setLength(sb.length() - 1); + } + return sb.toString(); + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/connection/AirflowConnectionCreator.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/connection/AirflowConnectionCreator.java new file mode 100644 index 00000000000..1e5b7d737cf --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/connection/AirflowConnectionCreator.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.api.connection; + +import org.apache.inlong.manager.pojo.schedule.airflow.AirflowConnection; +import org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant; +import org.apache.inlong.manager.schedule.airflow.api.BaseAirflowApi; +import org.apache.inlong.manager.schedule.exception.AirflowScheduleException; + +import lombok.extern.slf4j.Slf4j; +import okhttp3.MediaType; +import okhttp3.RequestBody; +import org.springframework.http.HttpMethod; + +import java.util.Map; + +import static org.apache.inlong.manager.schedule.exception.AirflowScheduleException.AirflowErrorCode.BUILD_REQUEST_BODY_FAILED; + +/** + * Build call for AirflowConnectionGetter
+ * + * + * + * + * + * + * + * + * + * + * + *
Request Body Param Description
connection_id The connection ID.
conn_type The connection type.
description The description of the connection.
host Host of the connection.
login Login of the connection.
schema Schema of the connection.
port Port of the connection.
password Password of the connection.
extra Other values that cannot be put into another field, e.g. RSA keys.(optional)
+ * + * @http.response.details + * + * + * + * + * + *
Status Code Description
200 Success.
400 Client specified an invalid argument.
401 Request not authenticated due to missing, invalid, authentication info.
403 Client does not have sufficient permission.
+ */ +@Slf4j +public class AirflowConnectionCreator extends BaseAirflowApi { + + AirflowConnection connection = null; + + public AirflowConnectionCreator(AirflowConnection connection) { + this.connection = connection; + } + + public AirflowConnectionCreator(Map requestBodyParams) { + this.requestBodyParams = requestBodyParams; + } + + @Override + public RequestBody getRequestBody() { + if (connection != null) { + try { + return RequestBody.create(MediaType.parse("application/json; charset=utf-8"), + objectMapper.writeValueAsString(connection)); + } catch (Exception e) { + log.error("Airflow request body construction failed: {}", e.getMessage(), e); + throw new AirflowScheduleException(BUILD_REQUEST_BODY_FAILED, + String.format("Airflow request body construction failed: %s", e.getMessage())); + } + } + return super.getRequestBody(); + } + + @Override + public Class getResponseType() { + return AirflowConnection.class; + } + + @Override + public HttpMethod getMethod() { + return HttpMethod.POST; + } + + @Override + public String getPath() { + return AirFlowAPIConstant.LIST_CONNECTIONS_URI; + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/connection/AirflowConnectionGetter.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/connection/AirflowConnectionGetter.java new file mode 100644 index 00000000000..7dc278ae0b3 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/connection/AirflowConnectionGetter.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.api.connection; + +import org.apache.inlong.manager.pojo.schedule.airflow.AirflowConnection; +import org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant; +import org.apache.inlong.manager.schedule.airflow.api.BaseAirflowApi; + +import org.springframework.http.HttpMethod; + +/** + * Build call for AirflowConnectionGetter
+ * + * + * + *
Path Param Description
connection_id The connection ID.
+ * + * @http.response.details + * + * + * + * + * + *
Status Code Description
200 Success.
401 Request not authenticated due to missing, invalid, authentication info.
403 Client does not have sufficient permission.
404 A specified resource is not found.
+ */ +public class AirflowConnectionGetter extends BaseAirflowApi { + + public AirflowConnectionGetter(String connectionId) { + pathParams.put("connection_id", connectionId); + } + + @Override + public HttpMethod getMethod() { + return HttpMethod.GET; + } + + @Override + public String getPath() { + return AirFlowAPIConstant.GET_CONNECTION_URI; + } + + @Override + public Class getResponseType() { + return AirflowConnection.class; + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dag/DAGCollectionUpdater.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dag/DAGCollectionUpdater.java new file mode 100644 index 00000000000..039a18d9d84 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dag/DAGCollectionUpdater.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.api.dag; + +import org.apache.inlong.manager.pojo.schedule.airflow.DAGCollection; +import org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant; +import org.apache.inlong.manager.schedule.airflow.api.BaseAirflowApi; + +import org.springframework.http.HttpMethod; + +import java.util.Map; + +/** + * Build call for DAGCollectionUpdater< br> + * + * + * + * + * + * + * + * + *
GET Param Description
limit The numbers of items to return. (optional, default to 100)
offset The number of items to skip before starting to collect the result set. (optional)
tags List of tags to filter results.(optional)
update_mask The fields to update on the resource. If absent or empty, all modifiable fields are updated. A comma-separated list of fully qualified names of fields.(optional)
only_active Only filter active DAGs. (optional, default to true)
dag_id_pattern If set, only return DAGs with dag_ids matching this pattern. (required)
+ * + * + * + * + *
Request Body Param Description
is_paused Whether the DAG is paused.
+ * + * @http.response.details + * + * + * + * + * + *
Status Code Description
200 Success.
401 Request not authenticated due to missing, invalid, authentication info.
403 Client does not have sufficient permission.
404 A specified resource is not found.
+ */ +public class DAGCollectionUpdater extends BaseAirflowApi { + + public DAGCollectionUpdater(String dagIdPattern, boolean isPaused) { + this.queryParams.put("dag_id_pattern", dagIdPattern); + this.requestBodyParams.put("is_paused", isPaused); + } + + public DAGCollectionUpdater(Map queryParams, Map requestBodyParams) { + this.queryParams = queryParams; + this.requestBodyParams = requestBodyParams; + } + + @Override + public HttpMethod getMethod() { + return HttpMethod.PATCH; + } + + @Override + public String getPath() { + return AirFlowAPIConstant.LIST_DAGS_URI; + } + + @Override + public Class getResponseType() { + return DAGCollection.class; + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dag/DAGDeletor.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dag/DAGDeletor.java new file mode 100644 index 00000000000..23a348d766a --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dag/DAGDeletor.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.api.dag; + +import org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant; +import org.apache.inlong.manager.schedule.airflow.api.BaseAirflowApi; + +import org.springframework.http.HttpMethod; + +/** + * Build call for DAGDeleter< br> + * + * + * + *
Path Param Description
dag_id The DAG ID.
+ * + * + * + * + *
GET Param Description
update_mask The fields to update on the resource. If absent or empty, all modifiable fields are updated. A comma-separated list of fully qualified names of fields.(optional)
+ * + * + * + * + *
Request Body Param Description
is_paused Whether the DAG is paused.
+ * + * @http.response.details + * + * + * + * + * + *
Status Code Description
200 Success.
401 Request not authenticated due to missing, invalid, authentication info.
403 Client does not have sufficient permission.
404 A specified resource is not found.
+ */ +public class DAGDeletor extends BaseAirflowApi { + + public DAGDeletor(String dagId) { + this.pathParams.put("dag_id", dagId); + } + @Override + public HttpMethod getMethod() { + return HttpMethod.DELETE; + } + + @Override + public String getPath() { + return AirFlowAPIConstant.UPDATE_DAG_URI; + } + + @Override + public Class getResponseType() { + return Object.class; + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dag/DAGUpdater.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dag/DAGUpdater.java new file mode 100644 index 00000000000..be8313f1b15 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dag/DAGUpdater.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.api.dag; + +import org.apache.inlong.manager.pojo.schedule.airflow.DAG; +import org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant; +import org.apache.inlong.manager.schedule.airflow.api.BaseAirflowApi; + +import org.springframework.http.HttpMethod; + +/** + * Build call for DAGUpdater< br> + * + * + * + *
Path Param Description
dag_id The DAG ID.
+ * + * + * + * + *
GET Param Description
update_mask The fields to update on the resource. If absent or empty, all modifiable fields are updated. A comma-separated list of fully qualified names of fields.(optional)
+ * + * + * + * + *
Request Body Param Description
is_paused Whether the DAG is paused.
+ * + * @http.response.details + * + * + * + * + * + *
Status Code Description
200 Success.
401 Request not authenticated due to missing, invalid, authentication info.
403 Client does not have sufficient permission.
404 A specified resource is not found.
+ */ +public class DAGUpdater extends BaseAirflowApi { + + public DAGUpdater(String dagId, boolean isPaused) { + this.pathParams.put("dag_id", dagId); + this.requestBodyParams.put("is_paused", isPaused); + } + + public DAGUpdater(String dagId, String updateMask, boolean isPaused) { + this.pathParams.put("dag_id", dagId); + this.queryParams.put("update_mask", updateMask); + this.requestBodyParams.put("is_paused", isPaused); + } + + @Override + public HttpMethod getMethod() { + return HttpMethod.PATCH; + } + + @Override + public String getPath() { + return AirFlowAPIConstant.UPDATE_DAG_URI; + } + + @Override + public Class getResponseType() { + return DAG.class; + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dagruns/DAGRunsTrigger.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dagruns/DAGRunsTrigger.java new file mode 100644 index 00000000000..b9fe7b22600 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/api/dagruns/DAGRunsTrigger.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.api.dagruns; + +import org.apache.inlong.manager.pojo.schedule.airflow.DAGRun; +import org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant; +import org.apache.inlong.manager.schedule.airflow.api.BaseAirflowApi; + +import org.springframework.http.HttpMethod; + +import java.util.Map; + +/** + * Build call for DAGRunsTrigger
+ * + * + * + *
Path Param Description
dag_id The DAG ID.
+ * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
Request Body Param Description
conf + * JSON object describing additional configuration parameters.
+ * The value of this field can be set only when creating the object. If you try to modify the field of an existing object, the request fails with an BAD_REQUEST error.
+ *
dag_run_id Run ID.
+ * The value of this field can be set only when creating the object. If you try to modify the field of an existing object, the request fails with an BAD_REQUEST error.
+ * If not provided, a value will be generated based on execution_date.
+ * If the specified dag_run_id is in use, the creation request fails with an ALREADY_EXISTS error.
+ * This together with DAG_ID are a unique key.
+ *
data_interval_end The end of the interval the DAG run covers.
data_interval_start The beginning of the interval the DAG run covers.
logical_date + * The logical date (previously called execution date). This is the time or interval covered by this DAG run, according to the DAG definition.
+ * The value of this field can be set only when creating the object. If you try to modify the field of an existing object, the request fails with an BAD_REQUEST error.
+ * This together with DAG_ID are a unique key.
+ *
note Contains manually entered notes by the user about the DagRun.
+ * + * @http.response.details + * + * + * + * + * + *
Status Code Description
200 Success.
401 Request not authenticated due to missing, invalid, authentication info.
403 Client does not have sufficient permission.
404 A specified resource is not found.
+ */ + +public class DAGRunsTrigger extends BaseAirflowApi { + + public DAGRunsTrigger(String dagId) { + this.pathParams.put("dag_id", dagId); + } + + public DAGRunsTrigger(String dagId, Map requestBodyParams) { + this.pathParams.put("dag_id", dagId); + this.requestBodyParams = requestBodyParams; + } + + @Override + public HttpMethod getMethod() { + return HttpMethod.POST; + } + + @Override + public String getPath() { + return AirFlowAPIConstant.TRIGGER_NEW_DAG_RUN_URI; + } + + @Override + public Class getResponseType() { + return DAGRun.class; + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/config/AirflowConfig.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/config/AirflowConfig.java new file mode 100644 index 00000000000..9e7ffbf9d03 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/config/AirflowConfig.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.config; + +import org.apache.inlong.manager.client.api.ClientConfiguration; +import org.apache.inlong.manager.schedule.airflow.AirflowServerClient; +import org.apache.inlong.manager.schedule.airflow.interceptor.AirflowAuthInterceptor; +import org.apache.inlong.manager.schedule.airflow.interceptor.LoggingInterceptor; + +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.EqualsAndHashCode; +import lombok.NoArgsConstructor; +import okhttp3.OkHttpClient; +import org.eclipse.jetty.util.StringUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; + +import javax.annotation.PostConstruct; + +import java.net.URL; + +@Data +@Configuration +@NoArgsConstructor +@AllArgsConstructor +@EqualsAndHashCode(callSuper = true) +public class AirflowConfig extends ClientConfiguration { + + private static final Logger LOGGER = LoggerFactory.getLogger(AirflowConfig.class); + @Value("${schedule.engine.inlong.manager.url:http://127.0.0.1:8083}") + private String inlongManagerUrl; + + private String inlongManagerHost; + private int inlongManagerPort; + + @Value("${default.admin.user:admin}") + private String inlongUsername; + + @Value("${default.admin.password:inlong}") + private String inlongPassword; + + @Value("${schedule.engine.airflow.connection.id:inlong_connection}") + private String connectionId; + + @Value("${schedule.engine.airflow.cleaner.id:dag_cleaner}") + private String dagCleanerId; + + @Value("${schedule.engine.airflow.creator.id:dag_creator}") + private String dagCreatorId; + + @Value("${schedule.engine.airflow.username:airflow}") + private String airflowUsername; + + @Value("${schedule.engine.airflow.password:airflow}") + private String airflowPassword; + + @Value("${schedule.engine.airflow.baseUrl:http://localhost:8080/}") + private String baseUrl; + + @PostConstruct + public void init() { + try { + if (StringUtil.isNotBlank(inlongManagerUrl)) { + URL url = new URL(inlongManagerUrl); + this.inlongManagerHost = url.getHost(); + this.inlongManagerPort = url.getPort(); + if (this.inlongManagerPort == -1) { + this.inlongManagerPort = 8083; + } + } + LOGGER.info("Init AirflowConfig success for manager url ={}", this.inlongManagerUrl); + } catch (Exception e) { + LOGGER.error("Init AirflowConfig failed for manager url={}: ", this.inlongManagerUrl, e); + } + } + + @Bean + public OkHttpClient okHttpClient() { + return new OkHttpClient.Builder() + .addInterceptor(new AirflowAuthInterceptor(this.getAirflowUsername(), this.getAirflowPassword())) + .addInterceptor(new LoggingInterceptor()) + .connectTimeout(this.getConnectTimeout(), this.getTimeUnit()) + .readTimeout(this.getReadTimeout(), this.getTimeUnit()) + .writeTimeout(this.getWriteTimeout(), this.getTimeUnit()) + .retryOnConnectionFailure(true) + .build(); + } + + @Bean + public AirflowServerClient airflowServerClient(OkHttpClient okHttpClient, AirflowConfig airflowConfig) { + return new AirflowServerClient(okHttpClient, airflowConfig); + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/interceptor/AirflowAuthInterceptor.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/interceptor/AirflowAuthInterceptor.java new file mode 100644 index 00000000000..714614bf949 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/interceptor/AirflowAuthInterceptor.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.interceptor; + +import lombok.extern.slf4j.Slf4j; +import okhttp3.Interceptor; +import okhttp3.Request; +import okhttp3.Response; + +import java.io.IOException; +import java.util.Base64; + +/** + * AirflowAuthInterceptor + * Before okhttp call a request, uniformly encapsulate the relevant parameters of authentication + */ +@Slf4j +public class AirflowAuthInterceptor implements Interceptor { + + // Airflow Authentication Header + private final String authHeader; + + public AirflowAuthInterceptor(String username, String password) { + String credentials = username + ":" + password; + this.authHeader = "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes()); + } + + @Override + public Response intercept(Chain chain) throws IOException { + Request originalRequest = chain.request(); + Request.Builder requestBuilder = originalRequest + .newBuilder() + .header("Authorization", authHeader); + return chain.proceed(requestBuilder.build()); + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/interceptor/LoggingInterceptor.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/interceptor/LoggingInterceptor.java new file mode 100644 index 00000000000..c3028385b18 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/interceptor/LoggingInterceptor.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.interceptor; + +import lombok.extern.slf4j.Slf4j; +import okhttp3.Interceptor; +import okhttp3.Request; +import okhttp3.Response; + +import java.io.IOException; + +/** + * LoggingInterceptor + * Provide unified logging for okhttp + */ +@Slf4j +public class LoggingInterceptor implements Interceptor { + + @Override + public Response intercept(Chain chain) throws IOException { + Request request = chain.request(); + Response response = chain.proceed(request); + log.info("Airflow API request information - Address: {}, URI: {}, Request method: {}, Response status code: {}", + request.url(), request.url().uri(), request.method(), response.code()); + return response; + } +} diff --git a/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/sink/DirtyServerType.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/util/DAGUtil.java similarity index 71% rename from inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/sink/DirtyServerType.java rename to inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/util/DAGUtil.java index 63f993c146a..fad05f2116c 100644 --- a/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/sink/DirtyServerType.java +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/util/DAGUtil.java @@ -15,23 +15,13 @@ * limitations under the License. */ -package org.apache.inlong.sort.base.dirty.sink; +package org.apache.inlong.manager.schedule.airflow.util; -public enum DirtyServerType { +import static org.apache.inlong.manager.schedule.airflow.AirFlowAPIConstant.INLONG_OFFLINE_DAG_TASK_PREFIX; - UNDEFINED("Undefined"), - TUBE_MQ("TubeMQ"), - ICEBERG("Iceberg") +public class DAGUtil { - ; - - private final String format; - - DirtyServerType(String format) { - this.format = format; - } - - public String format() { - return format; + public static String buildDAGIdByGroupId(String groupId) { + return INLONG_OFFLINE_DAG_TASK_PREFIX.concat(groupId); } } diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/util/DateUtil.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/util/DateUtil.java new file mode 100644 index 00000000000..950e334921c --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/airflow/util/DateUtil.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow.util; + +import org.apache.inlong.manager.schedule.ScheduleUnit; + +import java.math.BigInteger; +import java.util.Objects; + +public class DateUtil { + + public static String intervalToSeconds(long interval, String timeUnit) { + BigInteger seconds = new BigInteger(String.valueOf(interval)); + String intervalStr = ""; + switch (Objects.requireNonNull(ScheduleUnit.getScheduleUnit(timeUnit))) { + case SECOND: + intervalStr = "1"; + break; + case MINUTE: + intervalStr = "60"; + break; + case HOUR: + intervalStr = "3600"; + break; + case DAY: + intervalStr = "86400"; + break; + case WEEK: + intervalStr = "604800"; + break; + case MONTH: + intervalStr = "2592000"; + break; + case YEAR: + intervalStr = "31536000"; + break; + default: + throw new IllegalArgumentException("Unsupported time unit"); + } + return seconds.multiply(new BigInteger(intervalStr)).toString(); + } + +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleClient.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleClient.java new file mode 100644 index 00000000000..a75085b9ac9 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleClient.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.dolphinscheduler; + +import org.apache.inlong.manager.pojo.schedule.ScheduleInfo; +import org.apache.inlong.manager.schedule.ScheduleEngineClient; +import org.apache.inlong.manager.schedule.ScheduleEngineType; + +import org.springframework.stereotype.Service; + +import javax.annotation.Resource; + +/** + * Built-in implementation of third-party schedule engine client corresponding with {@link DolphinScheduleEngine}. + * DolphinScheduleClient simply invokes the {@link DolphinScheduleEngine} to register/unregister/update + * schedule info, all the logic for invoking the remote scheduling service is implemented in {@link DolphinScheduleEngine} + */ +@Service +public class DolphinScheduleClient implements ScheduleEngineClient { + + @Resource + public DolphinScheduleEngine scheduleEngine; + + @Override + public boolean accept(String engineType) { + return ScheduleEngineType.DOLPHINSCHEDULER.getType().equalsIgnoreCase(engineType); + } + + @Override + public boolean register(ScheduleInfo scheduleInfo) { + return scheduleEngine.handleRegister(scheduleInfo); + } + + @Override + public boolean unregister(String groupId) { + return scheduleEngine.handleUnregister(groupId); + } + + @Override + public boolean update(ScheduleInfo scheduleInfo) { + return scheduleEngine.handleUpdate(scheduleInfo); + } +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleConstants.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleConstants.java new file mode 100644 index 00000000000..1488ca1fe86 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleConstants.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.dolphinscheduler; + +public class DolphinScheduleConstants { + + // DS public constants + public static final String DS_ID = "id"; + public static final String DS_CODE = "code"; + public static final String DS_SUCCESS = "success"; + public static final String DS_TOKEN = "token"; + public static final String DS_PAGE_SIZE = "pageSize"; + public static final String DS_PAGE_NO = "pageNo"; + public static final String DS_SEARCH_VAL = "searchVal"; + public static final String DS_RESPONSE_DATA = "data"; + public static final String DS_RESPONSE_NAME = "name"; + public static final String DS_RESPONSE_TOTAL_LIST = "totalList"; + public static final int DS_DEFAULT_RETRY_TIMES = 3; + public static final int DS_DEFAULT_WAIT_MILLS = 1000; + public static final String DS_DEFAULT_PAGE_SIZE = "10"; + public static final String DS_DEFAULT_PAGE_NO = "1"; + public static final String DS_DEFAULT_TIMEZONE_ID = "Asia/Shanghai"; + + // DS project related constants + public static final String DS_PROJECT_URL = "/projects"; + public static final String DS_PROJECT_NAME = "projectName"; + public static final String DS_PROJECT_DESC = "description"; + public static final String DS_DEFAULT_PROJECT_NAME = "default_inlong_offline_scheduler"; + public static final String DS_DEFAULT_PROJECT_DESC = "default scheduler project for inlong offline job"; + + // DS task related constants + public static final String DS_TASK_CODE_URL = "/task-definition/gen-task-codes"; + public static final String DS_TASK_RELATION = "taskRelationJson"; + public static final String DS_TASK_DEFINITION = "taskDefinitionJson"; + public static final String DS_TASK_GEN_NUM = "genNum"; + public static final String DS_DEFAULT_TASK_GEN_NUM = "1"; + public static final String DS_DEFAULT_TASK_NAME = "default-inlong-http-callback"; + public static final String DS_DEFAULT_TASK_DESC = "default http request using shell script callbacks to inlong"; + + // DS process definition related constants + public static final String DS_PROCESS_URL = "/process-definition"; + public static final String DS_PROCESS_QUERY_URL = "/query-process-definition-list"; + public static final String DS_PROCESS_NAME = "name"; + public static final String DS_PROCESS_DESC = "description"; + public static final String DS_PROCESS_CODE = "processDefinitionCode"; + public static final String DS_DEFAULT_PROCESS_NAME = "_inlong_offline_process_definition"; + public static final String DS_DEFAULT_PROCESS_DESC = "scheduler process definition for inlong group: "; + + // DS release related constants + public static final String DS_RELEASE_URL = "/release"; + public static final String DS_RELEASE_STATE = "releaseState"; + + // DS schedule related constants + public static final String DS_SCHEDULE_URL = "/schedules"; + public static final String DS_SCHEDULE_DEF = "schedule"; + public static final String DS_DEFAULT_SCHEDULE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss"; + + // DS online/offline related constants + public static final String DS_ONLINE_URL = "/online"; + public static final String DS_ONLINE_STATE = "ONLINE"; + public static final String DS_OFFLINE_URL = "/offline"; + public static final String DS_OFFLINE_STATE = "OFFLINE"; + +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleEngine.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleEngine.java new file mode 100644 index 00000000000..c2d6ef00949 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleEngine.java @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.dolphinscheduler; + +import org.apache.inlong.manager.pojo.schedule.ScheduleInfo; +import org.apache.inlong.manager.schedule.ScheduleEngine; +import org.apache.inlong.manager.schedule.exception.DolphinScheduleException; + +import com.google.common.annotations.VisibleForTesting; +import lombok.Data; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Service; + +import javax.annotation.PostConstruct; +import javax.annotation.Resource; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_PROCESS_DESC; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_PROCESS_NAME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_PROJECT_DESC; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_PROJECT_NAME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_OFFLINE_STATE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_ONLINE_STATE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PROCESS_QUERY_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PROCESS_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PROJECT_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_SCHEDULE_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_TASK_CODE_URL; + +/** + * The default implementation of DolphinScheduler engine based on DolphinScheduler API. Response for processing + * the register/unregister/update requests from {@link DolphinScheduleClient} + */ +@Data +@Service +public class DolphinScheduleEngine implements ScheduleEngine { + + private static final Logger LOGGER = LoggerFactory.getLogger(DolphinScheduleEngine.class); + + @Value("${schedule.engine.inlong.manager.url:http://127.0.0.1:8083}") + private String inlongManagerUrl; + + @Value("${default.admin.user:admin}") + private String username; + + @Value("${default.admin.password:inlong}") + private String password; + + @Value("${schedule.engine.dolphinscheduler.url:http://127.0.0.1:12345/dolphinscheduler}") + private String dolphinUrl; + + @Value("${schedule.engine.dolphinscheduler.token:default_token_value}") + private String token; + + @Resource + private DolphinScheduleOperator dolphinScheduleOperator; + + private long projectCode; + private String projectBaseUrl; + private final Map scheduledProcessMap; + + @PostConstruct + public void init() { + this.projectBaseUrl = dolphinUrl + DS_PROJECT_URL; + } + + public DolphinScheduleEngine(String inlongManagerUrl, String username, String password, + String dolphinUrl, + String token) { + this.inlongManagerUrl = inlongManagerUrl; + this.username = username; + this.password = password; + this.dolphinUrl = dolphinUrl; + this.token = token; + this.scheduledProcessMap = new ConcurrentHashMap<>(); + } + + public DolphinScheduleEngine() { + this.scheduledProcessMap = new ConcurrentHashMap<>(); + } + + /** + * check if there already exists a project for inlong offline schedule + * if no then build a new project for inlong-group-id in DolphinScheduler + */ + @Override + public void start() { + LOGGER.info("Starting dolphin scheduler engine, Checking project exists..."); + long code = dolphinScheduleOperator.checkAndGetUniqueId(projectBaseUrl, token, DS_DEFAULT_PROJECT_NAME); + if (code != 0) { + LOGGER.info("Project exists, project code: {}", code); + this.projectCode = code; + + LOGGER.info("Starting synchronize existing process definition"); + String queryProcessDefUrl = projectBaseUrl + "/" + projectCode + DS_PROCESS_URL + DS_PROCESS_QUERY_URL; + scheduledProcessMap.putAll(dolphinScheduleOperator.queryAllProcessDef(queryProcessDefUrl, token)); + + } else { + LOGGER.info("There is no inlong offline project exists, default project will be created"); + this.projectCode = + dolphinScheduleOperator.creatProject(projectBaseUrl, token, DS_DEFAULT_PROJECT_NAME, + DS_DEFAULT_PROJECT_DESC); + } + } + + /** + * Handle schedule register. + * @param scheduleInfo schedule info to register + */ + @Override + @VisibleForTesting + public boolean handleRegister(ScheduleInfo scheduleInfo) { + start(); + String processDefUrl = projectBaseUrl + "/" + projectCode + DS_PROCESS_URL; + String scheduleUrl = projectBaseUrl + "/" + projectCode + DS_SCHEDULE_URL; + String processName = scheduleInfo.getInlongGroupId() + DS_DEFAULT_PROCESS_NAME; + String processDesc = DS_DEFAULT_PROCESS_DESC + scheduleInfo.getInlongGroupId(); + + LOGGER.info("Dolphin Scheduler handle register begin for {}, Checking process definition id uniqueness...", + scheduleInfo.getInlongGroupId()); + try { + long processDefCode = dolphinScheduleOperator.checkAndGetUniqueId(processDefUrl, token, processName); + + boolean online = false; + if (processDefCode != 0 || scheduledProcessMap.containsKey(processDefCode)) { + + // process definition already exists, delete and rebuild + LOGGER.info("Process definition exists, process definition id: {}, deleting...", processDefCode); + if (dolphinScheduleOperator.releaseProcessDef(processDefUrl, processDefCode, token, DS_OFFLINE_STATE)) { + dolphinScheduleOperator.deleteProcessDef(processDefUrl, token, processDefCode); + scheduledProcessMap.remove(processDefCode); + } + } + String taskCodeUrl = projectBaseUrl + "/" + projectCode + DS_TASK_CODE_URL; + + long taskCode = dolphinScheduleOperator.genTaskCode(taskCodeUrl, token); + LOGGER.info("Generate task code for process definition success, task code: {}", taskCode); + + long offset = DolphinScheduleUtils.calculateOffset(scheduleInfo); + processDefCode = + dolphinScheduleOperator.createProcessDef(processDefUrl, token, processName, processDesc, taskCode, + inlongManagerUrl, username, password, offset, scheduleInfo.getInlongGroupId()); + LOGGER.info("Create process definition success, process definition code: {}", processDefCode); + + if (dolphinScheduleOperator.releaseProcessDef(processDefUrl, processDefCode, token, DS_ONLINE_STATE)) { + LOGGER.info("Release process definition success, release status: {}", DS_ONLINE_STATE); + + int scheduleId = dolphinScheduleOperator.createScheduleForProcessDef(scheduleUrl, processDefCode, token, + scheduleInfo); + LOGGER.info("Create schedule for process definition success, schedule info: {}", scheduleInfo); + + online = dolphinScheduleOperator.onlineScheduleForProcessDef(scheduleUrl, scheduleId, token); + LOGGER.info("Online schedule for process definition, status: {}", online); + } + + scheduledProcessMap.putIfAbsent(processDefCode, processName); + return online; + } catch (Exception e) { + LOGGER.error("Failed to handle unregister dolphin scheduler: ", e); + throw new DolphinScheduleException( + String.format("Failed to handle unregister dolphin scheduler: %s", e.getMessage())); + } + } + + /** + * Handle schedule unregister. + * @param groupId group to un-register schedule info + */ + @Override + @VisibleForTesting + public boolean handleUnregister(String groupId) { + start(); + String processName = groupId + DS_DEFAULT_PROCESS_NAME; + String processDefUrl = projectBaseUrl + "/" + projectCode + DS_PROCESS_URL; + + LOGGER.info("Dolphin Scheduler handle Unregister begin for {}, Checking process definition id uniqueness...", + groupId); + try { + long processDefCode = dolphinScheduleOperator.checkAndGetUniqueId(processDefUrl, token, processName); + if (processDefCode != 0 || scheduledProcessMap.containsKey(processDefCode)) { + + LOGGER.info("Deleting process definition, process definition id: {}", processDefCode); + if (dolphinScheduleOperator.releaseProcessDef(processDefUrl, processDefCode, token, DS_OFFLINE_STATE)) { + + dolphinScheduleOperator.deleteProcessDef(processDefUrl, token, processDefCode); + scheduledProcessMap.remove(processDefCode); + LOGGER.info("Process definition deleted"); + } + } + LOGGER.info("Un-registered dolphin schedule info for {}", groupId); + return !scheduledProcessMap.containsKey(processDefCode); + } catch (Exception e) { + LOGGER.error("Failed to handle unregister dolphin scheduler: ", e); + throw new DolphinScheduleException( + String.format("Failed to handle unregister dolphin scheduler: %s", e.getMessage())); + } + } + + /** + * Handle schedule update. + * @param scheduleInfo schedule info to update + */ + @Override + @VisibleForTesting + public boolean handleUpdate(ScheduleInfo scheduleInfo) { + LOGGER.info("Update dolphin schedule info for {}", scheduleInfo.getInlongGroupId()); + try { + return handleUnregister(scheduleInfo.getInlongGroupId()) && handleRegister(scheduleInfo); + } catch (Exception e) { + LOGGER.error("Failed to handle update dolphin scheduler: ", e); + throw new DolphinScheduleException( + String.format("Failed to handle update dolphin scheduler: %s", e.getMessage())); + } + } + + /** + * stop and delete all process definition in DolphinScheduler + * remove all process stored in scheduledProcessMap + * delete project for inlong-group-id in DolphinScheduler + */ + @Override + public void stop() { + LOGGER.info("Stopping dolphin scheduler engine..."); + String processDefUrl = projectBaseUrl + "/" + projectCode + DS_PROCESS_URL; + try { + + String queryProcessDefUrl = projectBaseUrl + "/" + projectCode + DS_PROCESS_URL + DS_PROCESS_QUERY_URL; + Map allProcessDef = dolphinScheduleOperator.queryAllProcessDef(queryProcessDefUrl, token); + + for (Long processDefCode : allProcessDef.keySet()) { + + LOGGER.info("delete process definition id: {}", processDefCode); + dolphinScheduleOperator.releaseProcessDef(processDefUrl, processDefCode, token, DS_OFFLINE_STATE); + dolphinScheduleOperator.deleteProcessDef(processDefUrl, token, processDefCode); + scheduledProcessMap.remove(processDefCode); + } + + dolphinScheduleOperator.deleteProject(projectBaseUrl, token, projectCode); + LOGGER.info("Dolphin scheduler engine stopped"); + + } catch (Exception e) { + LOGGER.error("Failed to stop dolphin scheduler: ", e); + throw new DolphinScheduleException(String.format("Failed to stop dolphin scheduler: %s", e.getMessage())); + } + } + +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleOperator.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleOperator.java new file mode 100644 index 00000000000..8a7d9cbe2bd --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleOperator.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.dolphinscheduler; + +import org.apache.inlong.manager.pojo.schedule.ScheduleInfo; +import org.apache.inlong.manager.schedule.exception.DolphinScheduleException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.stereotype.Service; + +import java.util.Map; + +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.UNEXPECTED_ERROR; + +/** + * DolphinScheduler operator, This class includes methods for creating, updating, and deleting projects, + * tasks, and process definitions in DolphinScheduler. + */ +@Service +public class DolphinScheduleOperator { + + private static final Logger LOGGER = LoggerFactory.getLogger(DolphinScheduleOperator.class); + + /** + * Checks the uniqueness of a DolphinScheduler project ID based on the given search value. + */ + public long checkAndGetUniqueId(String url, String token, String searchVal) { + try { + return DolphinScheduleUtils.checkAndGetUniqueId(url, token, searchVal); + } catch (Exception e) { + LOGGER.error("Unexpected wrong in check id uniqueness: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected wrong in check id uniqueness: %s", e.getMessage())); + } + } + + /** + * Creates a new project in DolphinScheduler. + */ + public long creatProject(String url, String token, String projectName, String description) { + try { + return DolphinScheduleUtils.creatProject(url, token, projectName, description); + } catch (Exception e) { + LOGGER.error("Unexpected error while creating new project: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected error while creating new project: %s", e.getMessage())); + } + } + + /** + * Query all process definition in DolphinScheduler project. + */ + public Map queryAllProcessDef(String url, String token) { + try { + return DolphinScheduleUtils.queryAllProcessDef(url, token); + } catch (Exception e) { + LOGGER.error("Unexpected error while querying process definition: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected error while querying process definition: %s", e.getMessage())); + } + } + + /** + * Generates a new task code in DolphinScheduler. + */ + public long genTaskCode(String url, String token) { + try { + return DolphinScheduleUtils.genTaskCode(url, token); + } catch (Exception e) { + LOGGER.error("Unexpected wrong in generating task code: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected wrong in generating task code: %s", e.getMessage())); + } + } + + /** + * Creates a process definition in DolphinScheduler. + */ + public long createProcessDef(String url, String token, String name, String desc, long taskCode, + String inlongManagerUrl, String username, String password, long offset, String groupId) { + try { + return DolphinScheduleUtils.createProcessDef(url, token, name, desc, taskCode, inlongManagerUrl, username, + password, offset, groupId); + } catch (Exception e) { + LOGGER.error("Unexpected wrong in creating process definition: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected wrong in creating process definition: %s", e.getMessage())); + } + } + + /** + * Releases a process definition in DolphinScheduler. + */ + public boolean releaseProcessDef(String processDefUrl, long processDefCode, String token, String status) { + try { + return DolphinScheduleUtils.releaseProcessDef(processDefUrl, processDefCode, token, status); + } catch (Exception e) { + LOGGER.error("Unexpected wrong in release process definition: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected wrong in release process definition: %s", e.getMessage())); + } + } + + /** + * Create a schedule for process definition in DolphinScheduler. + */ + public int createScheduleForProcessDef(String url, long processDefCode, String token, ScheduleInfo scheduleInfo) { + try { + return DolphinScheduleUtils.createScheduleForProcessDef(url, processDefCode, token, + scheduleInfo); + } catch (Exception e) { + LOGGER.error("Unexpected wrong in creating schedule for process definition: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected wrong in creating schedule for process definition: %s", e.getMessage())); + } + } + + /** + * Online the schedule for process definition in DolphinScheduler. + */ + public boolean onlineScheduleForProcessDef(String scheduleUrl, int scheduleId, String token) { + try { + return DolphinScheduleUtils.onlineScheduleForProcessDef(scheduleUrl, scheduleId, token); + } catch (Exception e) { + LOGGER.error("Unexpected wrong in online process definition: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected wrong in online process definition: %s", e.getMessage())); + } + } + + /** + * Delete the process definition in DolphinScheduler. + */ + public void deleteProcessDef(String processDefUrl, String token, long processDefCode) { + try { + DolphinScheduleUtils.delete(processDefUrl, token, processDefCode); + } catch (Exception e) { + LOGGER.error("Unexpected wrong in deleting process definition: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected wrong in deleting process definition: %s", e.getMessage())); + } + } + + /** + * Delete the project in DolphinScheduler. + */ + public void deleteProject(String projectBaseUrl, String token, long projectCode) { + try { + DolphinScheduleUtils.delete(projectBaseUrl, token, projectCode); + } catch (Exception e) { + LOGGER.error("Unexpected wrong in deleting project definition: ", e); + throw new DolphinScheduleException(UNEXPECTED_ERROR, + String.format("Unexpected wrong in deleting project definition: %s", e.getMessage())); + } + } + +} diff --git a/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleUtils.java b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleUtils.java new file mode 100644 index 00000000000..ee28c6973f7 --- /dev/null +++ b/inlong-manager/manager-schedule/src/main/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleUtils.java @@ -0,0 +1,821 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.dolphinscheduler; + +import org.apache.inlong.common.bounded.BoundaryType; +import org.apache.inlong.manager.pojo.schedule.ScheduleInfo; +import org.apache.inlong.manager.pojo.schedule.dolphinschedule.DSTaskDefinition; +import org.apache.inlong.manager.pojo.schedule.dolphinschedule.DSTaskParams; +import org.apache.inlong.manager.pojo.schedule.dolphinschedule.DSTaskRelation; +import org.apache.inlong.manager.pojo.schedule.dolphinschedule.DScheduleInfo; +import org.apache.inlong.manager.schedule.ScheduleUnit; +import org.apache.inlong.manager.schedule.exception.DolphinScheduleException; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.JsonArray; +import com.google.gson.JsonElement; +import com.google.gson.JsonObject; +import com.google.gson.JsonParseException; +import com.google.gson.JsonParser; +import okhttp3.HttpUrl; +import okhttp3.MediaType; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.RequestBody; +import okhttp3.Response; +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.core.util.CronExpression; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_CODE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_PAGE_NO; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_PAGE_SIZE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_RETRY_TIMES; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_SCHEDULE_TIME_FORMAT; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_TASK_DESC; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_TASK_GEN_NUM; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_TASK_NAME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_TIMEZONE_ID; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_WAIT_MILLS; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_ID; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_ONLINE_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PAGE_NO; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PAGE_SIZE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PROCESS_CODE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PROCESS_DESC; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PROCESS_NAME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PROJECT_DESC; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_PROJECT_NAME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_RELEASE_STATE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_RELEASE_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_RESPONSE_DATA; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_RESPONSE_NAME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_RESPONSE_TOTAL_LIST; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_SCHEDULE_DEF; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_SEARCH_VAL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_SUCCESS; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_TASK_DEFINITION; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_TASK_GEN_NUM; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_TASK_RELATION; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_TOKEN; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.DELETION_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.GEN_TASK_CODE_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.HTTP_REQUEST_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.INVALID_HTTP_METHOD; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.JSON_PARSE_ERROR; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.NETWORK_ERROR; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.PROCESS_DEFINITION_CREATION_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.PROCESS_DEFINITION_IN_USED_ERROR; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.PROCESS_DEFINITION_QUERY_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.PROCESS_DEFINITION_RELEASE_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.PROJECT_CREATION_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.SCHEDULE_CREATION_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.SCHEDULE_ONLINE_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.UNEXPECTED_ERROR; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.UNIQUE_CHECK_FAILED; +import static org.apache.inlong.manager.schedule.exception.DolphinScheduleException.UNSUPPORTED_SCHEDULE_TYPE; + +/** + * DolphinScheduler utils + * A utility class for interacting with DolphinScheduler API. + */ +public class DolphinScheduleUtils { + + private static final Logger LOGGER = LoggerFactory.getLogger(DolphinScheduleEngine.class); + + private static final String POST = "POST"; + private static final String GET = "GET"; + private static final String DELETE = "DELETE"; + private static final long MILLIS_IN_SECOND = 1000L; + private static final long MILLIS_IN_MINUTE = 60 * MILLIS_IN_SECOND; + private static final long MILLIS_IN_HOUR = 60 * MILLIS_IN_MINUTE; + private static final long MILLIS_IN_DAY = 24 * MILLIS_IN_HOUR; + private static final long MILLIS_IN_WEEK = 7 * MILLIS_IN_DAY; + private static final long MILLIS_IN_MONTH = 30 * MILLIS_IN_DAY; + private static final long MILLIS_IN_YEAR = 365 * MILLIS_IN_DAY; + private static final String CONTENT_TYPE = "Content-Type: application/json; charset=utf-8"; + private static final String SHELL_REQUEST_API = "/inlong/manager/api/group/submitOfflineJob"; + private static final OkHttpClient CLIENT = new OkHttpClient(); + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private DolphinScheduleUtils() { + } + + /** + * Checks the uniqueness of a project ID based on the given search value. + * + * @param url The base URL of the DolphinScheduler API. + * @param token The authentication token to be used in the request header. + * @param searchVal The name of the project to search for. + * @return The unique project ID if found, or 0 if not found or an error occurs. + */ + public static long checkAndGetUniqueId(String url, String token, String searchVal) { + try { + Map header = buildHeader(token); + Map queryParams = buildPageParam(searchVal); + + JsonObject response = executeHttpRequest(url, GET, queryParams, header); + + JsonObject data = response.get(DS_RESPONSE_DATA).getAsJsonObject(); + JsonArray totalList = data.getAsJsonArray(DS_RESPONSE_TOTAL_LIST); + + // check uniqueness + if (totalList != null && totalList.size() == 1) { + JsonObject project = totalList.get(0).getAsJsonObject(); + String name = project.get(DS_RESPONSE_NAME).getAsString(); + if (name.equals(searchVal)) { + return project.get(DS_CODE).getAsLong(); + } + } + return 0; + } catch (JsonParseException e) { + LOGGER.error("JsonParseException during checkAndGetUniqueId", e); + throw new DolphinScheduleException(JSON_PARSE_ERROR, + String.format("Error parsing json during unique ID check for: %s at URL: %s", searchVal, url), e); + + } catch (DolphinScheduleException e) { + LOGGER.error("DolphinScheduleException during unique ID check: {}", e.getDetailedMessage(), e); + throw new DolphinScheduleException(UNIQUE_CHECK_FAILED, + String.format("Error checking unique ID for %s at URL: %s", searchVal, url)); + } + } + + /** + * Creates a new project in DolphinScheduler. + * + * @param url The base URL of the DolphinScheduler API. + * @param token The authentication token to be used in the request header. + * @param projectName The name of the new project. + * @param description The description of the new project. + * @return The project code (ID) if creation is successful, or 0 if an error occurs. + */ + public static long creatProject(String url, String token, String projectName, + String description) { + try { + Map header = buildHeader(token); + + Map queryParams = new HashMap<>(); + queryParams.put(DS_PROJECT_NAME, projectName); + queryParams.put(DS_PROJECT_DESC, description); + + JsonObject response = executeHttpRequest(url, POST, queryParams, header); + + JsonObject data = response.get(DS_RESPONSE_DATA).getAsJsonObject(); + LOGGER.info("create project success, project data: {}", data); + + return data != null ? data.get(DS_CODE).getAsLong() : 0; + } catch (JsonParseException e) { + LOGGER.error("JsonParseException during creating project", e); + throw new DolphinScheduleException( + JSON_PARSE_ERROR, + String.format("Error creating project with name: %s and description: %s at URL: %s", + projectName, description, url)); + + } catch (DolphinScheduleException e) { + LOGGER.error("Creating project failed: {}", e.getMessage()); + throw new DolphinScheduleException( + PROJECT_CREATION_FAILED, + String.format("Error creating project with name: %s and description: %s at URL: %s", + projectName, description, url)); + } + } + + /** + * Query all process definition in project + * + * @param url The base URL of the DolphinScheduler API. + * @param token The authentication token to be used in the request header. + * @return Map of all the process definition + */ + public static Map queryAllProcessDef(String url, String token) { + Map header = buildHeader(token); + try { + JsonObject response = executeHttpRequest(url, GET, new HashMap<>(), header); + + Map processDef = + StreamSupport.stream(response.get(DS_RESPONSE_DATA).getAsJsonArray().spliterator(), false) + .map(JsonElement::getAsJsonObject) + .collect(Collectors.toMap( + jsonObject -> jsonObject.get(DS_CODE).getAsLong(), + jsonObject -> jsonObject.get(DS_PROCESS_NAME).getAsString())); + + LOGGER.info("Query all process definition success, processes info: {}", processDef); + return processDef; + + } catch (JsonParseException e) { + LOGGER.error("JsonParseException during query all process definition", e); + throw new DolphinScheduleException( + JSON_PARSE_ERROR, + String.format("Error querying all process definitions at URL: %s", url)); + + } catch (DolphinScheduleException e) { + LOGGER.info("Query all process definition failed: {}", e.getMessage()); + throw new DolphinScheduleException( + PROCESS_DEFINITION_QUERY_FAILED, + String.format("Error querying all process definitions at URL: %s", url)); + } + + } + + /** + * Generates a new task code in DolphinScheduler. + * + * @param url The base URL of the DolphinScheduler API. + * @param token The authentication token to be used in the request header. + * @return The task code (ID) if generation is successful, or 0 if an error occurs. + */ + public static long genTaskCode(String url, String token) { + try { + Map header = buildHeader(token); + + Map queryParams = new HashMap<>(); + queryParams.put(DS_TASK_GEN_NUM, DS_DEFAULT_TASK_GEN_NUM); + + JsonObject response = executeHttpRequest(url, GET, queryParams, header); + + JsonArray data = response.get(DS_RESPONSE_DATA).getAsJsonArray(); + + LOGGER.info("Generate task code success, task code data: {}", data); + return data != null && data.size() == 1 ? data.get(0).getAsLong() : 0; + + } catch (JsonParseException e) { + LOGGER.error("JsonParseException during generate task code", e); + throw new DolphinScheduleException( + JSON_PARSE_ERROR, + String.format("Error generate task code at URL: %s", url)); + + } catch (DolphinScheduleException e) { + LOGGER.info("generate task code failed: {}", e.getMessage()); + throw new DolphinScheduleException( + GEN_TASK_CODE_FAILED, + String.format("Error generate task code at URL: %s", url)); + } + } + + /** + * Creates a process definition in DolphinScheduler. + * + * @param url The base URL of the DolphinScheduler API. + * @param token The authentication token to be used in the request header. + * @param name The name of the process definition. + * @param desc The description of the process definition. + * @param taskCode The task code to be associated with this process definition. + * @param inlongManagerUrl The host where the process will run. + * @param username The username for authentication. + * @param password The password for authentication. + * @param offset The offset for the scheduling. + * @param groupId The group ID of the process. + * @return The process definition code (ID) if creation is successful, or 0 if an error occurs. + */ + public static long createProcessDef(String url, String token, String name, String desc, + long taskCode, String inlongManagerUrl, String username, String password, long offset, String groupId) + throws Exception { + try { + Map header = buildHeader(token); + + DSTaskRelation taskRelation = new DSTaskRelation(); + taskRelation.setPostTaskCode(taskCode); + String taskRelationJson = MAPPER.writeValueAsString(Collections.singletonList(taskRelation)); + + DSTaskParams taskParams = new DSTaskParams(); + taskParams.setRawScript(buildScript(inlongManagerUrl, username, password, offset, groupId)); + + DSTaskDefinition taskDefinition = new DSTaskDefinition(); + taskDefinition.setCode(taskCode); + taskDefinition.setName(DS_DEFAULT_TASK_NAME); + taskDefinition.setDescription(DS_DEFAULT_TASK_DESC); + taskDefinition.setTaskParams(taskParams); + String taskDefinitionJson = MAPPER.writeValueAsString(Collections.singletonList(taskDefinition)); + + Map queryParams = new HashMap<>(); + queryParams.put(DS_TASK_RELATION, taskRelationJson); + queryParams.put(DS_TASK_DEFINITION, taskDefinitionJson); + queryParams.put(DS_PROCESS_NAME, name); + queryParams.put(DS_PROCESS_DESC, desc); + + JsonObject data = executeHttpRequest(url, POST, queryParams, header); + + LOGGER.info("create process definition success, process definition data: {}", data); + return data != null ? data.get(DS_RESPONSE_DATA).getAsJsonObject().get(DS_CODE).getAsLong() : 0; + } catch (JsonParseException e) { + LOGGER.error("JsonParseException during creating process definition", e); + throw new DolphinScheduleException( + JSON_PARSE_ERROR, + String.format("Error creating process definition with name: %s and description: %s at URL: %s", + name, desc, url)); + + } catch (DolphinScheduleException e) { + throw new DolphinScheduleException( + PROCESS_DEFINITION_CREATION_FAILED, + String.format("Error creating process definition with name: %s and description: %s at URL: %s", + name, desc, url)); + } + } + + /** + * Releases a process definition in DolphinScheduler. + * + * @param processDefUrl The URL to release the process definition. + * @param processDefCode The ID of the process definition. + * @param token The authentication token to be used in the request header. + * @param status The status to set for the process definition (e.g., "online" or "offline"). + * @return true if the process definition was successfully released, false otherwise. + */ + public static boolean releaseProcessDef(String processDefUrl, long processDefCode, + String token, String status) { + try { + String url = processDefUrl + "/" + processDefCode + DS_RELEASE_URL; + Map header = buildHeader(token); + + Map queryParam = new HashMap<>(); + queryParam.put(DS_RELEASE_STATE, status); + + JsonObject response = executeHttpRequest(url, POST, queryParam, header); + LOGGER.info("release process definition success, response data: {}", response); + + return response.get(DS_RESPONSE_DATA).getAsBoolean(); + + } catch (JsonParseException e) { + LOGGER.error("JsonParseException during releasing process definition", e); + throw new DolphinScheduleException( + JSON_PARSE_ERROR, + String.format("Error releasing process definition with code: %d and status: %s at URL: %s", + processDefCode, status, processDefUrl)); + + } catch (DolphinScheduleException e) { + throw new DolphinScheduleException( + PROCESS_DEFINITION_RELEASE_FAILED, + String.format("Error releasing process definition with code: %d and status: %s at URL: %s", + processDefCode, status, processDefUrl)); + } + } + + /** + * Create a schedule for process definition in DolphinScheduler. + * + * @param url The URL to create a schedule for the process definition. + * @param processDefCode The ID of the process definition. + * @param token The authentication token to be used in the request header. + * @param scheduleInfo The schedule info + * @return The schedule id + */ + public static int createScheduleForProcessDef(String url, long processDefCode, + String token, ScheduleInfo scheduleInfo) throws Exception { + + try { + Map header = buildHeader(token); + + DateTimeFormatter formatter = DateTimeFormatter.ofPattern(DS_DEFAULT_SCHEDULE_TIME_FORMAT); + String startTime = scheduleInfo.getStartTime().toLocalDateTime() + .atZone(ZoneId.of(DS_DEFAULT_TIMEZONE_ID)).format(formatter); + String endTime = scheduleInfo.getEndTime().toLocalDateTime() + .atZone(ZoneId.of(DS_DEFAULT_TIMEZONE_ID)).format(formatter); + + String crontab; + switch (scheduleInfo.getScheduleType()) { + case 0: + crontab = generateCrontabExpression(scheduleInfo.getScheduleUnit(), + scheduleInfo.getScheduleInterval()); + break; + + case 1: + crontab = scheduleInfo.getCrontabExpression(); + break; + + default: + LOGGER.error("Unsupported schedule type: {}", scheduleInfo.getScheduleType()); + throw new DolphinScheduleException("Unsupported schedule type: " + scheduleInfo.getScheduleType()); + } + + DScheduleInfo dScheduleInfo = new DScheduleInfo(); + dScheduleInfo.setStartTime(startTime); + dScheduleInfo.setEndTime(endTime); + dScheduleInfo.setCrontab(crontab); + dScheduleInfo.setTimezoneId(DS_DEFAULT_TIMEZONE_ID); + String scheduleDef = MAPPER.writeValueAsString(dScheduleInfo); + + Map queryParams = new HashMap<>(); + queryParams.put(DS_PROCESS_CODE, String.valueOf(processDefCode)); + queryParams.put(DS_SCHEDULE_DEF, scheduleDef); + + JsonObject response = executeHttpRequest(url, POST, queryParams, header); + LOGGER.info("create schedule for process definition success, response data: {}", response); + + return response.get(DS_RESPONSE_DATA).getAsJsonObject().get(DS_ID).getAsInt(); + + } catch (JsonParseException e) { + LOGGER.error("JsonParseException during releasing process definition", e); + throw new DolphinScheduleException( + JSON_PARSE_ERROR, + String.format("Error creating schedule for process definition code: %d at URL: %s", + processDefCode, url)); + + } catch (DolphinScheduleException e) { + throw new DolphinScheduleException( + SCHEDULE_CREATION_FAILED, + String.format("Error creating schedule for process definition code: %d at URL: %s", + processDefCode, url)); + } + } + + /** + * Online the schedule for process definition in DolphinScheduler. + * + * @param scheduleUrl The URL to online the schedule for process definition. + * @param scheduleId The ID of the schedule of process definition. + * @param token The authentication token to be used in the request header. + * @return whether online is succeeded + */ + public static boolean onlineScheduleForProcessDef(String scheduleUrl, int scheduleId, + String token) { + try { + Map header = buildHeader(token); + + String url = scheduleUrl + "/" + scheduleId + DS_ONLINE_URL; + JsonObject response = executeHttpRequest(url, POST, new HashMap<>(), header); + LOGGER.info("online schedule for process definition success, response data: {}", response); + + if (response != null && !response.get(DS_RESPONSE_DATA).isJsonNull()) { + return response.get(DS_RESPONSE_DATA).getAsBoolean(); + } + return false; + + } catch (JsonParseException e) { + LOGGER.error("JsonParseException during online schedule", e); + throw new DolphinScheduleException( + JSON_PARSE_ERROR, + String.format("Error online schedule with ID: %d online at URL: %s", scheduleId, scheduleUrl)); + + } catch (DolphinScheduleException e) { + throw new DolphinScheduleException( + SCHEDULE_ONLINE_FAILED, + String.format("Error online schedule with ID: %d online at URL: %s", scheduleId, scheduleUrl)); + } + } + + /** + * Delete the process definition in DolphinScheduler. + * + * @param url The URL to delete the project or process definition. + * @param token The authentication token to be used in the request header. + * @param code The project code or process definition code + */ + public static void delete(String url, String token, long code) { + try { + Map header = buildHeader(token); + + String requestUrl = url + "/" + code; + for (int retryTime = 1; retryTime <= DS_DEFAULT_RETRY_TIMES; retryTime++) { + JsonObject response = executeHttpRequest(requestUrl, DELETE, new HashMap<>(), header); + if (response.get(DS_CODE).getAsInt() == PROCESS_DEFINITION_IN_USED_ERROR) { + + LOGGER.warn( + "Retrying for current retry time ={}, maximum retry count={}, code={}, url={}, after {} ms...", + retryTime, DS_DEFAULT_RETRY_TIMES, code, url, DS_DEFAULT_WAIT_MILLS); + Thread.sleep(DS_DEFAULT_WAIT_MILLS); + + } else if (response.get(DS_SUCCESS).getAsBoolean()) { + LOGGER.info("Delete process or project success, response data: {}", response); + return; + } else { + LOGGER.warn("Delete process or project failed, response data: {}", response); + } + + } + LOGGER.error( + "Maximum retry attempts reached for deleting process or project. URL: {}, Code: {}", + url, code); + throw new DolphinScheduleException( + DELETION_FAILED, + String.format("Failed to delete after %d retries. Code: %d at URL: %s", + DS_DEFAULT_RETRY_TIMES, code, url)); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.error("Thread interrupted while retrying delete process or project: ", e); + throw new DolphinScheduleException( + DELETION_FAILED, + String.format("Thread interrupted while retrying delete for code: %d at URL: %s", code, url)); + } catch (JsonParseException e) { + LOGGER.error("JsonParseException during deleting process or project", e); + throw new DolphinScheduleException( + JSON_PARSE_ERROR, + String.format("Error deleting process or project with code: %d at URL: %s", code, url)); + + } catch (DolphinScheduleException e) { + LOGGER.error("Error deleting process or project for code={}, url={} ", code, url, e); + throw new DolphinScheduleException( + DELETION_FAILED, + String.format("Error deleting process or project with code: %d at URL: %s", code, url)); + } + } + + /** + * Builds the header map for HTTP requests, including the authentication token. + * + * @param token The authentication token for the request. + * @return A map representing the headers of the HTTP request. + */ + private static Map buildHeader(String token) { + Map headers = new HashMap<>(); + if (StringUtils.isNotEmpty(token)) { + headers.put(DS_TOKEN, token); + } + return headers; + } + + /** + * Builds a query parameter map used for API calls that need to paginate or filter results. + * This method can be used for searching projects or tasks. + * + * @param searchVal The value to search for. + * @return A map containing the necessary query parameters. + */ + private static Map buildPageParam(String searchVal) { + Map queryParams = new HashMap<>(); + queryParams.put(DS_SEARCH_VAL, searchVal); + queryParams.put(DS_PAGE_SIZE, DS_DEFAULT_PAGE_SIZE); + queryParams.put(DS_PAGE_NO, DS_DEFAULT_PAGE_NO); + return queryParams; + } + + /** + * Calculate the offset according to schedule info + * + * @param scheduleInfo The schedule info + * @return timestamp between two schedule task + */ + public static long calculateOffset(ScheduleInfo scheduleInfo) { + if (scheduleInfo == null) { + LOGGER.error("ScheduleInfo cannot be null"); + throw new DolphinScheduleException("ScheduleInfo cannot be null"); + } + + long offset = 0; + + // Determine offset based on schedule type + if (scheduleInfo.getScheduleType() == null) { + LOGGER.error("Schedule type cannot be null"); + throw new DolphinScheduleException("Schedule type cannot be null"); + } + + switch (scheduleInfo.getScheduleType()) { + case 0: // Normal scheduling + offset = calculateNormalOffset(scheduleInfo); + break; + case 1: // Crontab scheduling + offset = calculateCronOffset(scheduleInfo); + break; + default: + LOGGER.error("Invalid schedule type"); + throw new DolphinScheduleException( + UNSUPPORTED_SCHEDULE_TYPE, "Invalid schedule type"); + } + + // Add delay time if specified + if (scheduleInfo.getDelayTime() != null) { + offset += scheduleInfo.getDelayTime() * MILLIS_IN_SECOND; + } + + return offset; + } + + private static long calculateNormalOffset(ScheduleInfo scheduleInfo) { + if (scheduleInfo.getScheduleInterval() == null || scheduleInfo.getScheduleUnit() == null) { + LOGGER.error("Schedule interval and unit cannot be null for normal scheduling"); + throw new IllegalArgumentException("Schedule interval and unit cannot be null for normal scheduling"); + } + switch (Objects.requireNonNull(ScheduleUnit.getScheduleUnit(scheduleInfo.getScheduleUnit()))) { + case YEAR: + return scheduleInfo.getScheduleInterval() * MILLIS_IN_YEAR; + case MONTH: + return scheduleInfo.getScheduleInterval() * MILLIS_IN_MONTH; + case WEEK: + return scheduleInfo.getScheduleInterval() * MILLIS_IN_WEEK; + case DAY: + return scheduleInfo.getScheduleInterval() * MILLIS_IN_DAY; + case HOUR: + return scheduleInfo.getScheduleInterval() * MILLIS_IN_HOUR; + case MINUTE: + return scheduleInfo.getScheduleInterval() * MILLIS_IN_MINUTE; + case SECOND: + return scheduleInfo.getScheduleInterval() * MILLIS_IN_SECOND; + case ONE_ROUND: + return scheduleInfo.getScheduleInterval(); + default: + LOGGER.error("Invalid schedule unit"); + throw new DolphinScheduleException("Invalid schedule unit"); + } + } + + private static long calculateCronOffset(ScheduleInfo scheduleInfo) { + if (scheduleInfo.getCrontabExpression() == null) { + LOGGER.error("Crontab expression cannot be null for schedule type crontab"); + throw new DolphinScheduleException("Crontab expression cannot be null for schedule type crontab"); + } + + try { + CronExpression cronExpression = new CronExpression(scheduleInfo.getCrontabExpression()); + Date firstExecution = cronExpression.getNextValidTimeAfter(new Date()); + Date secondExecution = cronExpression.getNextValidTimeAfter(firstExecution); + + if (secondExecution != null) { + return secondExecution.getTime() - firstExecution.getTime(); + } else { + LOGGER.error("Unable to calculate the next execution times for the cron expression"); + throw new DolphinScheduleException( + "Unable to calculate the next execution times for the cron expression"); + } + } catch (Exception e) { + LOGGER.error("Invalid cron expression: ", e); + throw new DolphinScheduleException(String.format("Invalid cron expression: %s", e.getMessage())); + } + } + + private static String generateCrontabExpression(String scheduleUnit, Integer scheduleInterval) { + if (scheduleUnit.isEmpty()) { + LOGGER.error("Schedule unit and interval must not be null for generating crontab expression"); + throw new DolphinScheduleException( + "Schedule unit and interval must not be null for generating crontab expression"); + } + String crontabExpression; + + switch (Objects.requireNonNull(ScheduleUnit.getScheduleUnit(scheduleUnit))) { + case SECOND: + crontabExpression = String.format("0/%d * * * * ? *", scheduleInterval); + break; + case MINUTE: + crontabExpression = String.format("* 0/%d * * * ? *", scheduleInterval); + break; + case HOUR: + crontabExpression = String.format("* * 0/%d * * ? *", scheduleInterval); + break; + case DAY: + crontabExpression = String.format("* * * 1/%d * ? *", scheduleInterval); + break; + case WEEK: + crontabExpression = String.format("* * * 1/%d * ? *", scheduleInterval * 7); + break; + case MONTH: + crontabExpression = String.format("* * * * 0/%d ? *", scheduleInterval); + break; + case YEAR: + crontabExpression = String.format("* * * * * ? 0/%d", scheduleInterval); + break; + default: + LOGGER.error("Unsupported schedule unit for generating crontab: {}", scheduleUnit); + throw new DolphinScheduleException("Unsupported schedule unit for generating crontab: " + scheduleUnit); + } + + return crontabExpression; + } + + /** + * Executes an HTTP request using OkHttp. Supports various HTTP methods (GET, POST, PUT, DELETE). + * + * @param url The URL of the request. + * @param method The HTTP method (GET, POST, PUT, DELETE). + * @param queryParams The query parameters for the request (optional). + * @param headers The headers for the request. + * @return A JsonObject containing the response from the server. + * @throws DolphinScheduleException If an error occurs during the request. + */ + private static JsonObject executeHttpRequest(String url, String method, Map queryParams, + Map headers) { + HttpUrl.Builder urlBuilder = Objects.requireNonNull(HttpUrl.parse(url)).newBuilder(); + + for (Map.Entry entry : queryParams.entrySet()) { + urlBuilder.addQueryParameter(entry.getKey(), entry.getValue()); + } + HttpUrl httpUrl = urlBuilder.build(); + + Request.Builder requestBuilder = new Request.Builder() + .url(httpUrl); + + for (Map.Entry entry : headers.entrySet()) { + requestBuilder.addHeader(entry.getKey(), entry.getValue()); + } + RequestBody body = RequestBody.create(MediaType.parse(CONTENT_TYPE), ""); + + switch (method.toUpperCase()) { + case POST: + requestBuilder.post(body); + break; + case GET: + requestBuilder.get(); + break; + case DELETE: + requestBuilder.delete(body); + break; + default: + throw new DolphinScheduleException(INVALID_HTTP_METHOD, + String.format("Unsupported request method: %s", method)); + } + + Request request = requestBuilder.build(); + + // get response + try (Response response = CLIENT.newCall(request).execute()) { + String responseBody = response.body() != null ? response.body().string() : null; + LOGGER.debug("HTTP request to {} completed with status code {}", httpUrl, response.code()); + + if (response.isSuccessful() && responseBody != null) { + return JsonParser.parseString(responseBody).getAsJsonObject(); + } else { + LOGGER.error("HTTP request to {} failed. HTTP Status: {}, Response Body: {}", httpUrl, response.code(), + responseBody != null ? responseBody : "No response body"); + + throw new DolphinScheduleException( + HTTP_REQUEST_FAILED, + String.format("HTTP request to %s failed. Status: %d, Response: %s", + httpUrl, response.code(), responseBody != null ? responseBody : "No response body")); + } + } catch (IOException e) { + throw new DolphinScheduleException( + NETWORK_ERROR, + String.format("Network error during HTTP request to %s. Reason: %s", httpUrl, e.getMessage())); + } catch (Exception e) { + throw new DolphinScheduleException( + UNEXPECTED_ERROR, + String.format("Unexpected error during HTTP request to %s. Reason: %s", httpUrl, e.getMessage())); + } + } + + /** + * Shell node in DolphinScheduler need to write in a script + * When process definition schedule run, the shell node run, + * Call back in inlong, sending a request with parameters required + */ + private static String buildScript(String inlongManagerUrl, String username, String password, long offset, + String groupId) { + LOGGER.info("build script for Inlong Manager Url: {}, username: {}, password: {}, offset: {}, groupId: {}", + inlongManagerUrl, username, password, offset, groupId); + return "#!/bin/bash\n\n" + + + // Get current timestamp + "# Get current timestamp\n" + + "lowerBoundary=$(date +%s)\n" + + "echo \"get lowerBoundary: ${lowerBoundary}\"\n" + + "upperBoundary=$(($lowerBoundary + " + offset + "))\n" + + "echo \"get upperBoundary: ${upperBoundary}\"\n\n" + + + // Set URL + "# Set URL and HTTP method\n" + + "url=\"" + inlongManagerUrl + SHELL_REQUEST_API + + "?username=" + username + "&password=" + password + "\"\n" + + "echo \"get url: ${url}\"\n" + + + // Set HTTP method + "httpMethod=\"POST\"\n\n" + + + // Set request body + "# Build request body\n" + + "jsonBody=$(cat < environment; + private static OkHttpClient httpClient = new OkHttpClient(); + + public static void setUp() { + // Step 1: Start only the airflow-init service + environment = new DockerComposeContainer<>(new File(DOCKER_COMPOSE_YAML_PATH)) + .withServices("airflow-init") + .withEnv("AIRFLOW_UID", "$(id -u)"); + // Start the environment + environment.start(); + // Step 2: Wait until the "airflow-init" service has completed initialization + // Once initialized, stop the init-only environment and start the full environment + environment.stop(); + // Step 3: Start all services in detached mode after initialization + environment = new DockerComposeContainer<>(new File(DOCKER_COMPOSE_YAML_PATH)) + .withEnv("AIRFLOW_UID", "0") + .withEnv("AIRFLOW__CORE__LOAD_EXAMPLES", "false") + .withEnv("AIRFLOW__API__AUTH_BACKEND", + "airflow.providers.fab.auth_manager.api.auth.backend.basic_auth"); + environment.start(); + copyTestDAGs(); + waitForDAGsLoad("dag_cleaner"); + log.info("Airflow runtime environment created successfully."); + } + + public static void shutDown() { + if (environment != null) { + environment.stop(); + } + } + + private static void copyTestDAGs() { + // After the DAG file is created, the scheduler will regularly scan the DAG file directory and + // then load it into memory for scheduling. In order to quickly test the update and unregister, two + // test DAGs need to be loaded at the beginning. + Optional container = environment.getContainerByServiceName(AIRFLOW_SCHEDULER_CONTAINER_NAME); + if (container.isPresent()) { + ContainerState airflowScheduler = container.get(); + Path dagPath1 = Paths.get("src/test/resources/airflow/dag_cleaner.py").toAbsolutePath(); + Path dagPath2 = Paths.get("src/test/resources/airflow/dag_creator.py").toAbsolutePath(); + Path dagPath3 = Paths.get("src/test/resources/airflow/testGroup_cron.py").toAbsolutePath(); + Path dagPath4 = Paths.get("src/test/resources/airflow/testGroup_normal.py").toAbsolutePath(); + airflowScheduler.copyFileToContainer(MountableFile.forHostPath(dagPath1), + DEFAULT_DAGS_PATH.concat("dag_cleaner.py")); + airflowScheduler.copyFileToContainer(MountableFile.forHostPath(dagPath2), + DEFAULT_DAGS_PATH.concat("dag_creator.py")); + airflowScheduler.copyFileToContainer(MountableFile.forHostPath(dagPath3), + DEFAULT_DAGS_PATH.concat("testGroup_cron.py")); + airflowScheduler.copyFileToContainer(MountableFile.forHostPath(dagPath4), + DEFAULT_DAGS_PATH.concat("testGroup_normal.py")); + try { + String result = + airflowScheduler.execInContainer("bash", "-c", "ls ".concat(DEFAULT_DAGS_PATH)).getStdout(); + log.info(DEFAULT_DAGS_PATH.concat(" has file: {}"), result); + } catch (Exception e) { + log.warn(String.format( + "Copying the test DAG file may have failed. Docker Container command(\"%s\") execution failed.", + "ls ".contains(DEFAULT_DAGS_PATH)), e); + } + } else { + log.error(String.format("Copying test DAG file failed. Airflow scheduler container(%s) does not exist.", + AIRFLOW_SCHEDULER_CONTAINER_NAME)); + throw new AirflowScheduleException("Copying test DAG file failed."); + } + log.info("Copy test DAG file successfully."); + } + + public static void waitForDAGsLoad(String dagId) { + int total = 10; + // Waiting for Airflow to load the initial DAG + while (total > 0) { + String credential = okhttp3.Credentials.basic(AIRFLOW_USERNAME, AIRFLOW_PASSWORD); + Request request = new Request.Builder() + .url(BASE_URL + "/api/v1/dags/" + dagId + "/details") + .header("Authorization", credential) + .build(); + try (Response response = httpClient.newCall(request).execute()) { + if (response.code() == 200) { + break; + } + } catch (Exception e) { + log.error("The request to check if the original DAG exists failed: {}", e.getMessage(), e); + } + try { + Thread.sleep(30000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + total--; + } + log.info("DAG successfully loaded."); + } +} diff --git a/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/airflow/AirflowScheduleEngineTest.java b/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/airflow/AirflowScheduleEngineTest.java new file mode 100644 index 00000000000..fe5d070afdd --- /dev/null +++ b/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/airflow/AirflowScheduleEngineTest.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.airflow; + +import org.apache.inlong.manager.pojo.schedule.ScheduleInfo; +import org.apache.inlong.manager.schedule.BaseScheduleTest; + +import lombok.extern.slf4j.Slf4j; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.boot.context.properties.EnableConfigurationProperties; +import org.springframework.boot.test.context.SpringBootTest; +import org.springframework.context.annotation.ComponentScan; + +import static org.apache.inlong.manager.schedule.airflow.AirflowContainerEnv.CORN_POSTFIX; +import static org.apache.inlong.manager.schedule.airflow.AirflowContainerEnv.NORMAL_POSTFIX; +import static org.junit.jupiter.api.Assertions.assertTrue; + +@Slf4j +@EnableConfigurationProperties +@ComponentScan(basePackages = "org.apache.inlong.manager") +@SpringBootTest(classes = AirflowScheduleEngineTest.class) +public class AirflowScheduleEngineTest { + + @Autowired + private AirflowScheduleEngine scheduleEngine; + private static BaseScheduleTest baseScheduleTest = new BaseScheduleTest(); + + @BeforeAll + public static void initScheduleEngine() { + try { + AirflowContainerEnv.setUp(); + } catch (Exception e) { + log.error("Airflow runtime environment creation failed.", e); + throw new RuntimeException( + String.format("Airflow runtime environment creation failed: %s", e.getMessage())); + } + } + + @AfterAll + public static void stopScheduleEngine() { + AirflowContainerEnv.shutDown(); + } + + @Test + @Order(1) + public void testRegisterScheduleInfo() { + // 1. test for normal schedule + ScheduleInfo scheduleInfo = baseScheduleTest.genDefaultScheduleInfo(); + String groupId = scheduleInfo.getInlongGroupId() + NORMAL_POSTFIX + System.currentTimeMillis(); + scheduleInfo.setInlongGroupId(groupId); + assertTrue(scheduleEngine.handleRegister(scheduleInfo)); + + // 2. test for cron schedule + scheduleInfo = baseScheduleTest.genDefaultCronScheduleInfo(); + groupId = scheduleInfo.getInlongGroupId() + CORN_POSTFIX + System.currentTimeMillis(); + scheduleInfo.setInlongGroupId(groupId); + assertTrue(scheduleEngine.handleRegister(scheduleInfo)); + } + + @Test + @Order(2) + public void testUpdateScheduleInfo() { + // 1. test for normal schedule + ScheduleInfo scheduleInfo = baseScheduleTest.genDefaultScheduleInfo(); + String groupId = scheduleInfo.getInlongGroupId() + NORMAL_POSTFIX; + scheduleInfo.setInlongGroupId(groupId); + assertTrue(scheduleEngine.handleUpdate(scheduleInfo)); + + // 2. test for cron schedule, gen cron schedule info, */2 * * * * ? + scheduleInfo = baseScheduleTest.genDefaultCronScheduleInfo(); + groupId = scheduleInfo.getInlongGroupId() + CORN_POSTFIX; + scheduleInfo.setInlongGroupId(groupId); + assertTrue(scheduleEngine.handleUpdate(scheduleInfo)); + } + + @Test + @Order(3) + public void testUnRegisterScheduleInfo() { + // 1. test for normal schedule + ScheduleInfo scheduleInfo = baseScheduleTest.genDefaultScheduleInfo(); + String groupId = scheduleInfo.getInlongGroupId() + NORMAL_POSTFIX; + scheduleInfo.setInlongGroupId(groupId); + assertTrue(scheduleEngine.handleUnregister(scheduleInfo.getInlongGroupId())); + + // 2. test for cron schedule, gen cron schedule info, */2 * * * * ? + scheduleInfo = baseScheduleTest.genDefaultCronScheduleInfo(); + groupId = scheduleInfo.getInlongGroupId() + CORN_POSTFIX; + scheduleInfo.setInlongGroupId(groupId); + assertTrue(scheduleEngine.handleUnregister(scheduleInfo.getInlongGroupId())); + } +} diff --git a/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleContainerTestEnv.java b/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleContainerTestEnv.java new file mode 100644 index 00000000000..c57bd6e783d --- /dev/null +++ b/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleContainerTestEnv.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.dolphinscheduler; + +import org.apache.inlong.manager.schedule.BaseScheduleTest; +import org.apache.inlong.manager.schedule.exception.DolphinScheduleException; + +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import okhttp3.HttpUrl; +import okhttp3.MediaType; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.RequestBody; +import okhttp3.Response; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.output.Slf4jLogConsumer; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +import static org.apache.http.HttpHeaders.CONTENT_TYPE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_DEFAULT_TIMEZONE_ID; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinScheduleConstants.DS_RESPONSE_DATA; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_COOKIE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_COOKIE_SC_TYPE; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_COOKIE_SESSION_ID; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_DEFAULT_PASSWORD; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_DEFAULT_SERVICE_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_DEFAULT_USERID; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_DEFAULT_USERNAME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_EXPIRE_TIME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_EXPIRE_TIME_FORMAT; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_IMAGE_NAME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_IMAGE_TAG; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_LOGIN_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_PASSWORD; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_RESPONSE_TOKEN; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_TOKEN_GEN_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_TOKEN_URL; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_USERID; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.DS_USERNAME; +import static org.apache.inlong.manager.schedule.dolphinscheduler.DolphinSchedulerContainerEnvConstants.INTER_CONTAINER_DS_ALIAS; + +public abstract class DolphinScheduleContainerTestEnv extends BaseScheduleTest { + + private static final Logger DS_LOG = LoggerFactory.getLogger(DolphinScheduleEngineTest.class); + + private static final Network NETWORK = Network.newNetwork(); + + protected static final GenericContainer dolphinSchedulerContainer = + new GenericContainer<>(DS_IMAGE_NAME + ":" + DS_IMAGE_TAG) + .withExposedPorts(12345, 25333) + .withEnv("TZ", DS_DEFAULT_TIMEZONE_ID) + .withNetwork(NETWORK) + .withAccessToHost(true) + .withNetworkAliases(INTER_CONTAINER_DS_ALIAS) + .withLogConsumer(new Slf4jLogConsumer(DS_LOG)); + + /** + * This method just for DS testing, login by default admin username and password + * generate a 1-day expiring token for test, the token will disappear with the DS container shutting down + * + * @return the DS token + */ + protected static String accessToken() { + Map loginParams = new HashMap<>(); + loginParams.put(DS_USERNAME, DS_DEFAULT_USERNAME); + loginParams.put(DS_PASSWORD, DS_DEFAULT_PASSWORD); + try { + JsonObject loginResponse = + executeHttpRequest(DS_DEFAULT_SERVICE_URL + DS_LOGIN_URL, loginParams, new HashMap<>()); + if (loginResponse.get("success").getAsBoolean()) { + String tokenGenUrl = DS_DEFAULT_SERVICE_URL + DS_TOKEN_URL + DS_TOKEN_GEN_URL; + Map tokenParams = new HashMap<>(); + tokenParams.put(DS_USERID, String.valueOf(DS_DEFAULT_USERID)); + + LocalDateTime now = LocalDateTime.now(); + LocalDateTime tomorrow = now.plusDays(1); + DateTimeFormatter formatter = DateTimeFormatter.ofPattern(DS_EXPIRE_TIME_FORMAT); + String expireTime = tomorrow.format(formatter); + tokenParams.put(DS_EXPIRE_TIME, expireTime); + + Map cookies = new HashMap<>(); + cookies.put(DS_COOKIE_SC_TYPE, loginResponse.get(DS_RESPONSE_DATA) + .getAsJsonObject().get(DS_COOKIE_SC_TYPE).getAsString()); + cookies.put(DS_COOKIE_SESSION_ID, loginResponse.get(DS_RESPONSE_DATA) + .getAsJsonObject().get(DS_COOKIE_SESSION_ID).getAsString()); + + JsonObject tokenGenResponse = executeHttpRequest(tokenGenUrl, tokenParams, cookies); + + String accessTokenUrl = DS_DEFAULT_SERVICE_URL + DS_TOKEN_URL; + tokenParams.put(DS_RESPONSE_TOKEN, tokenGenResponse.get(DS_RESPONSE_DATA).getAsString()); + JsonObject result = executeHttpRequest(accessTokenUrl, tokenParams, cookies); + String token = result.get(DS_RESPONSE_DATA).getAsJsonObject().get(DS_RESPONSE_TOKEN).getAsString(); + DS_LOG.info("login and generate token success, token: {}", token); + return token; + } + return null; + } catch (Exception e) { + DS_LOG.error("login and generate token fail: ", e); + throw new DolphinScheduleException(String.format("login and generate token fail: %s", e.getMessage())); + } + } + + private static JsonObject executeHttpRequest(String url, Map queryParams, + Map cookies) throws IOException { + OkHttpClient client = new OkHttpClient(); + + // Build query parameters + HttpUrl.Builder urlBuilder = Objects.requireNonNull(HttpUrl.parse(url)).newBuilder(); + for (Map.Entry entry : queryParams.entrySet()) { + urlBuilder.addQueryParameter(entry.getKey(), entry.getValue()); + } + HttpUrl httpUrl = urlBuilder.build(); + + // Build the request + Request.Builder requestBuilder = new Request.Builder() + .url(httpUrl); + + // Add cookies to the request + if (cookies != null && !cookies.isEmpty()) { + String cookieHeader = cookies.entrySet() + .stream() + .map(entry -> entry.getKey() + "=" + entry.getValue()) + .collect(Collectors.joining("; ")); + requestBuilder.header(DS_COOKIE, cookieHeader); + } + + RequestBody body = RequestBody.create(MediaType.parse(CONTENT_TYPE), ""); + requestBuilder.post(body); + + Request request = requestBuilder.build(); + + // Execute the request and parse the response + try (Response response = client.newCall(request).execute()) { + if (response.isSuccessful() && response.body() != null) { + String responseBody = response.body().string(); + return JsonParser.parseString(responseBody).getAsJsonObject(); + } else { + DS_LOG.error("Unexpected http response error: {}", response); + throw new DolphinScheduleException("Unexpected http response error " + response); + } + } + } + +} diff --git a/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleEngineTest.java b/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleEngineTest.java new file mode 100644 index 00000000000..b63b04e7364 --- /dev/null +++ b/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinScheduleEngineTest.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.dolphinscheduler; + +import org.apache.inlong.manager.pojo.schedule.ScheduleInfo; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.MethodOrderer; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.TestMethodOrder; +import org.junit.jupiter.api.Timeout; +import org.springframework.boot.test.context.SpringBootTest; +import org.springframework.context.annotation.ComponentScan; + +import javax.annotation.Resource; + +import java.util.Arrays; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +@SpringBootTest(classes = DolphinScheduleEngineTest.class) +@ComponentScan(basePackages = "org.apache.inlong.manager") +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public class DolphinScheduleEngineTest extends DolphinScheduleContainerTestEnv { + + @Resource + private DolphinScheduleEngine dolphinScheduleEngine; + + @BeforeAll + public void beforeAll() { + dolphinSchedulerContainer.setPortBindings(Arrays.asList("12345:12345", "25333:25333")); + dolphinSchedulerContainer.start(); + assertTrue(dolphinSchedulerContainer.isRunning(), "DolphinScheduler container should be running"); + + String token = accessToken(); + dolphinScheduleEngine.setToken(token); + } + + @AfterAll + public void afterAll() { + dolphinScheduleEngine.stop(); + if (dolphinSchedulerContainer != null) { + dolphinSchedulerContainer.stop(); + } + } + + @Test + @Order(1) + @Timeout(30) + public void testRegisterScheduleInfo() { + // 1. test for normal schedule + ScheduleInfo scheduleInfo = genDefaultScheduleInfo(); + testRegister(scheduleInfo); + + // 2. test for cron schedule + scheduleInfo = genDefaultCronScheduleInfo(); + testRegister(scheduleInfo); + } + + private void testRegister(ScheduleInfo scheduleInfo) { + // register schedule info + dolphinScheduleEngine.handleRegister(scheduleInfo); + assertEquals(1, dolphinScheduleEngine.getScheduledProcessMap().size()); + } + + @Test + @Order(2) + @Timeout(30) + public void testUnRegisterScheduleInfo() { + // 1. test for normal schedule + ScheduleInfo scheduleInfo = genDefaultScheduleInfo(); + testUnRegister(scheduleInfo); + + // 2. test for cron schedule, gen cron schedule info, */2 * * * * ? + scheduleInfo = genDefaultCronScheduleInfo(); + testUnRegister(scheduleInfo); + } + + private void testUnRegister(ScheduleInfo scheduleInfo) { + // register schedule info + dolphinScheduleEngine.handleRegister(scheduleInfo); + assertEquals(1, dolphinScheduleEngine.getScheduledProcessMap().size()); + + // Un-register schedule info + dolphinScheduleEngine.handleUnregister(scheduleInfo.getInlongGroupId()); + assertEquals(0, dolphinScheduleEngine.getScheduledProcessMap().size()); + } + + @Test + @Order(3) + @Timeout(30) + public void testUpdateScheduleInfo() { + // 1. test for normal schedule + ScheduleInfo scheduleInfo = genDefaultScheduleInfo(); + testRegister(scheduleInfo); + + // 2. test for cron schedule, gen cron schedule info, */2 * * * * ? + scheduleInfo = genDefaultCronScheduleInfo(); + testUpdate(scheduleInfo); + } + + private void testUpdate(ScheduleInfo scheduleInfo) { + // register schedule info + dolphinScheduleEngine.handleUpdate(scheduleInfo); + assertEquals(1, dolphinScheduleEngine.getScheduledProcessMap().size()); + } +} diff --git a/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinSchedulerContainerEnvConstants.java b/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinSchedulerContainerEnvConstants.java new file mode 100644 index 00000000000..a2f6d97e0cd --- /dev/null +++ b/inlong-manager/manager-schedule/src/test/java/org/apache/inlong/manager/schedule/dolphinscheduler/DolphinSchedulerContainerEnvConstants.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.schedule.dolphinscheduler; + +public class DolphinSchedulerContainerEnvConstants { + + // DS env image related constants + protected static final String DS_IMAGE_NAME = "apache/dolphinscheduler-standalone-server"; + protected static final String DS_IMAGE_TAG = "3.2.2"; + protected static final String INTER_CONTAINER_DS_ALIAS = "dolphinscheduler"; + + // DS env url related constants + protected static final String DS_DEFAULT_SERVICE_URL = "http://127.0.0.1:12345/dolphinscheduler"; + protected static final String DS_LOGIN_URL = "/login"; + protected static final String DS_TOKEN_URL = "/access-tokens"; + protected static final String DS_TOKEN_GEN_URL = "/generate"; + + // DS env api params related constants + protected static final String DS_USERNAME = "userName"; + protected static final String DS_PASSWORD = "userPassword"; + protected static final String DS_USERID = "userId"; + protected static final String DS_COOKIE = "Cookie"; + protected static final String DS_COOKIE_SC_TYPE = "securityConfigType"; + protected static final String DS_COOKIE_SESSION_ID = "sessionId"; + protected static final String DS_EXPIRE_TIME = "expireTime"; + protected static final String DS_EXPIRE_TIME_FORMAT = "yyyy-MM-dd HH:mm:ss"; + + // DS env token related constants + protected static final String DS_RESPONSE_TOKEN = "token"; + + // DS env default admin user info + protected static final String DS_DEFAULT_USERNAME = "admin"; + protected static final String DS_DEFAULT_PASSWORD = "dolphinscheduler123"; + protected static final Integer DS_DEFAULT_USERID = 1; + +} diff --git a/inlong-manager/manager-schedule/src/test/resources/airflow/dag_cleaner.py b/inlong-manager/manager-schedule/src/test/resources/airflow/dag_cleaner.py new file mode 100644 index 00000000000..a24c5bebe49 --- /dev/null +++ b/inlong-manager/manager-schedule/src/test/resources/airflow/dag_cleaner.py @@ -0,0 +1,93 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow import DAG +from datetime import datetime, timedelta +from airflow.operators.python_operator import PythonOperator +from airflow.models import Variable +from airflow.utils.dates import days_ago +from datetime import datetime +import os +import logging +import pytz +from croniter import croniter +from airflow.hooks.base_hook import BaseHook +from airflow import configuration + +DAG_PATH = configuration.get('core', 'dags_folder') + "/" +DAG_PREFIX = 'inlong_offline_task_' + +def clean_expired_dags(**context): + + original_time = context.get('execution_date') + target_timezone = pytz.timezone("Asia/Shanghai") + utc_time = original_time.astimezone(target_timezone) + current_time = utc_time.strftime("%Y-%m-%d %H:%M:%S.%f") + + conf = context.get('dag_run').conf + groupId = conf.get('inlong_group_id') + + logging.info(f"Execution parameters = {conf} for groupId = {groupId} and execution time = {current_time}") + + if groupId is None or len(groupId) == 0: + for dag_file in os.listdir(DAG_PATH): + if dag_file.endswith(".py") and dag_file.startswith(DAG_PREFIX): + dag_file_path = os.path.join(DAG_PATH, dag_file) + with open(dag_file_path, "r") as file: + line = file.readline() + while line and "end_offset_datetime_str" not in line: + line = file.readline() + end_date_str = None + row = line.split("=") + if len(row) > 1: + end_date_str = datetime.fromtimestamp(int(row[1].strip().strip("\"")) / 1000, tz=target_timezone) + logging.info(f"The end time of {dag_file} is {end_date_str} for groupId = {dag_file.lstrip(DAG_PREFIX)}") + try: + if end_date_str and str(current_time) > str(end_date_str): + os.remove(dag_file_path) + logging.info(f"Deleted expired DAG: {dag_file} for groupId = {dag_file.lstrip(DAG_PREFIX)}") + except ValueError: + logging.error(f"Failed to delete {dag_file} for groupId = {dag_file.lstrip(DAG_PREFIX)}") + else: + dag_file = groupId + '.py' + if not str(groupId).startswith(DAG_PREFIX): + dag_file = DAG_PREFIX + dag_file + os.remove(os.path.join(DAG_PATH, dag_file)) + logging.info(f"Deleted expired DAG: {dag_file} for groupId = {groupId}") + + + +default_args = { + 'owner': 'airflow', + 'start_date': datetime.now() - timedelta(minutes=5), + 'catchup': False, + 'tags': ["inlong"] +} + +dag = DAG( + 'dag_cleaner', + default_args=default_args, + schedule_interval="*/20 * * * *", + is_paused_upon_creation=False +) + +clean_task = PythonOperator( + task_id='clean_expired_dags', + python_callable=clean_expired_dags, + provide_context=True, + dag=dag, +) diff --git a/inlong-manager/manager-schedule/src/test/resources/airflow/dag_creator.py b/inlong-manager/manager-schedule/src/test/resources/airflow/dag_creator.py new file mode 100644 index 00000000000..1958ab594a1 --- /dev/null +++ b/inlong-manager/manager-schedule/src/test/resources/airflow/dag_creator.py @@ -0,0 +1,149 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow import DAG +from airflow.operators.python_operator import PythonOperator +from airflow.utils.dates import days_ago +from airflow.models import Variable +import os +from datetime import datetime +from airflow.hooks.base_hook import BaseHook +from airflow import configuration + +DAG_PATH = configuration.get('core', 'dags_folder') + "/" +DAG_PREFIX = 'inlong_offline_task_' + +def create_dag_file(**context): + conf = context.get('dag_run').conf + print('conf: ', conf) + groupId = conf.get('inlong_group_id') + task_name = DAG_PREFIX + groupId + timezone = conf.get('timezone') + boundaryType = str(conf.get('boundary_type')) + start_time = int(conf.get('start_time')) + end_time = int(conf.get('end_time')) + cron_expr = conf.get('cron_expr') + seconds_interval = conf.get('seconds_interval') + schedule_interval = cron_expr + if cron_expr is None or len(cron_expr) == 0: + schedule_interval = f'timedelta(seconds={seconds_interval})' + else: + schedule_interval = '"' + cron_expr + '"' + connectionId = conf.get('connection_id') + dag_content = f'''from airflow import DAG +from datetime import datetime, timedelta +from airflow.operators.python_operator import PythonOperator +from datetime import datetime +from croniter import croniter +from airflow.hooks.base_hook import BaseHook +import requests +import pytz + +timezone = "{timezone}" +start_offset_datetime_str = {start_time} +end_offset_datetime_str = {end_time} +schedule_interval = {schedule_interval} # Or put cron expression +dag_id = "{task_name}" +groupId = "{groupId}" +connectionId = "{connectionId}" +boundaryType = "{boundaryType}" + +target_timezone = pytz.timezone(timezone) # Specify the time zone as China Standard Time + +start_date = datetime.fromtimestamp(start_offset_datetime_str / 1000, tz=target_timezone) +end_date = datetime.fromtimestamp(end_offset_datetime_str / 1000, tz=target_timezone) + +def taskFunction(**context): + print("#########################") + conn = BaseHook.get_connection(connectionId) + url = f"http://{{conn.host}}:{{conn.port}}/{{conn.schema}}" + params = {{ + "username": conn.login, + "password": conn.password + }} + headers = {{ + "User-Agent": "Mozilla/5.0 (Windows NT 10.0; Win64; x64; rv:131.0) Gecko/20100101 Firefox/131.0", + "Accept": "application/json", + "Accept-Language": "zh-CN,zh;q=0.8,zh-TW;q=0.7,zh-HK;q=0.5,en-US;q=0.3,en;q=0.2", + "Accept-Encoding": "gzip, deflate", + "Content-Type": "application/json;charset=UTF-8", + "tenant": "public", + "Connection": "close", + "Priority": "u=0" + }} + time_interval = get_time_interval(context) + data = {{ + "boundaryType": boundaryType, + "groupId": groupId, + "lowerBoundary": str(int(time_interval[0])), + "upperBoundary": str(int(int(time_interval[1]))) + }} + print("Connection ID: ", connectionId) + print("url: ", url) + print("params: ", params) + print("Request Body: ", data) + response = requests.post(url, params=params, headers=headers, json=data) + print("Response Code: ", response.status_code) + if response.status_code == 200: + print(response.json()) + else: + print(response.text) + print("#########################") + + +def get_time_interval(context): + execution_date = context.get('execution_date') + execution_date = execution_date.astimezone(target_timezone) + dag = context.get('dag') + schedule_interval = dag.schedule_interval + if isinstance(schedule_interval, timedelta): + return execution_date.timestamp(), (execution_date + schedule_interval).timestamp() + else: + cron_expr = dag.schedule_interval + cron = croniter(cron_expr, execution_date) + next_run = cron.get_next(datetime) + return execution_date.timestamp(), next_run.timestamp() + + +default_args = {{ + 'owner': 'inlong', + 'start_date': start_date, + 'end_date': end_date, + 'catchup': False, +}} + +dag = DAG( + dag_id, + default_args=default_args, + schedule_interval=schedule_interval, + is_paused_upon_creation=False +) + +clean_task = PythonOperator( + task_id=dag_id, + python_callable=taskFunction, + provide_context=True, + dag=dag, +) + ''' + dag_file_path = os.path.join(DAG_PATH, f'{task_name}.py') + with open(dag_file_path, 'w') as f: + f.write(dag_content) + print(f'Generated DAG file: {dag_file_path}') +default_args = {'owner': 'airflow', 'start_date': days_ago(1), 'catchup': False} +dag = DAG('dag_creator', default_args=default_args, schedule_interval=None, is_paused_upon_creation=False) +create_dag_task = PythonOperator(task_id='create_dag_file', python_callable=create_dag_file, provide_context=True, dag=dag) \ No newline at end of file diff --git a/inlong-manager/manager-schedule/src/test/resources/airflow/docker-compose.yaml b/inlong-manager/manager-schedule/src/test/resources/airflow/docker-compose.yaml new file mode 100644 index 00000000000..c97195c03f7 --- /dev/null +++ b/inlong-manager/manager-schedule/src/test/resources/airflow/docker-compose.yaml @@ -0,0 +1,292 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# Basic Airflow cluster configuration for CeleryExecutor with Redis and PostgreSQL. +# +# WARNING: This configuration is for local development. Do not use it in a production deployment. +# +# This configuration supports basic configuration using environment variables or an .env file +# The following variables are supported: +# +# AIRFLOW_IMAGE_NAME - Docker image name used to run Airflow. +# Default: apache/airflow:2.6.0 +# AIRFLOW_UID - User ID in Airflow containers +# Default: 50000 +# AIRFLOW_PROJ_DIR - Base path to which all the files will be volumed. +# Default: . +# Those configurations are useful mostly in case of standalone testing/running Airflow in test/try-out mode +# +# _AIRFLOW_WWW_USER_USERNAME - Username for the administrator account (if requested). +# Default: airflow +# _AIRFLOW_WWW_USER_PASSWORD - Password for the administrator account (if requested). +# Default: airflow +# _PIP_ADDITIONAL_REQUIREMENTS - Additional PIP requirements to add when starting all containers. +# Use this option ONLY for quick checks. Installing requirements at container +# startup is done EVERY TIME the service is started. +# A better way is to build a custom image or extend the official image +# as described in https://airflow.apache.org/docs/docker-stack/build.html. +# Default: '' +# +# Feel free to modify this file to suit your needs. +--- +version: '3.8' +x-airflow-common: + &airflow-common + # In order to add custom dependencies or upgrade provider packages you can use your extended image. + # Comment the image line, place your Dockerfile in the directory where you placed the docker-compose.yaml + # and uncomment the "build" line below, Then run `docker-compose build` to build the images. + image: ${AIRFLOW_IMAGE_NAME:-apache/airflow:2.6.0} + # build: . + environment: + &airflow-common-env + AIRFLOW__CORE__EXECUTOR: CeleryExecutor + AIRFLOW__DATABASE__SQL_ALCHEMY_CONN: postgresql+psycopg2://airflow:airflow@postgres/airflow + # For backward compatibility, with Airflow <2.3 + AIRFLOW__CORE__SQL_ALCHEMY_CONN: postgresql+psycopg2://airflow:airflow@postgres/airflow + AIRFLOW__CELERY__RESULT_BACKEND: db+postgresql://airflow:airflow@postgres/airflow + AIRFLOW__CELERY__BROKER_URL: redis://:@redis:6379/0 + AIRFLOW__CORE__FERNET_KEY: '' + AIRFLOW__CORE__DAGS_ARE_PAUSED_AT_CREATION: 'true' + AIRFLOW__CORE__LOAD_EXAMPLES: 'true' + AIRFLOW__API__AUTH_BACKENDS: 'airflow.api.auth.backend.basic_auth,airflow.api.auth.backend.session' + # yamllint disable rule:line-length + # Use simple http server on scheduler for health checks + # See https://airflow.apache.org/docs/apache-airflow/stable/administration-and-deployment/logging-monitoring/check-health.html#scheduler-health-check-server + # yamllint enable rule:line-length + AIRFLOW__SCHEDULER__ENABLE_HEALTH_CHECK: 'true' + # WARNING: Use _PIP_ADDITIONAL_REQUIREMENTS option ONLY for a quick checks + # for other purpose (development, test and especially production usage) build/extend Airflow image. + _PIP_ADDITIONAL_REQUIREMENTS: ${_PIP_ADDITIONAL_REQUIREMENTS:-} + user: "${AIRFLOW_UID:-50000}:0" + depends_on: + &airflow-common-depends-on + redis: + condition: service_healthy + postgres: + condition: service_healthy + +services: + postgres: + image: postgres:13 + environment: + POSTGRES_USER: airflow + POSTGRES_PASSWORD: airflow + POSTGRES_DB: airflow + volumes: + - postgres-db-volume:/var/lib/postgresql/data + healthcheck: + test: ["CMD", "pg_isready", "-U", "airflow"] + interval: 10s + retries: 5 + start_period: 5s + restart: always + + redis: + image: redis:latest + expose: + - 6379 + healthcheck: + test: ["CMD", "redis-cli", "ping"] + interval: 10s + timeout: 30s + retries: 50 + start_period: 30s + restart: always + + airflow-webserver: + <<: *airflow-common + command: webserver + ports: + - "8080:8080" + healthcheck: + test: ["CMD", "curl", "--fail", "http://localhost:8080/health"] + interval: 30s + timeout: 10s + retries: 5 + start_period: 30s + restart: always + depends_on: + <<: *airflow-common-depends-on + airflow-init: + condition: service_completed_successfully + + airflow-scheduler: + <<: *airflow-common + command: scheduler + healthcheck: + test: ["CMD", "curl", "--fail", "http://localhost:8974/health"] + interval: 30s + timeout: 10s + retries: 5 + start_period: 30s + restart: always + depends_on: + <<: *airflow-common-depends-on + airflow-init: + condition: service_completed_successfully + + airflow-worker: + <<: *airflow-common + command: celery worker + healthcheck: + test: + - "CMD-SHELL" + - 'celery --app airflow.executors.celery_executor.app inspect ping -d "celery@$${HOSTNAME}"' + interval: 30s + timeout: 10s + retries: 5 + start_period: 30s + environment: + <<: *airflow-common-env + # Required to handle warm shutdown of the celery workers properly + # See https://airflow.apache.org/docs/docker-stack/entrypoint.html#signal-propagation + DUMB_INIT_SETSID: "0" + restart: always + depends_on: + <<: *airflow-common-depends-on + airflow-init: + condition: service_completed_successfully + + airflow-triggerer: + <<: *airflow-common + command: triggerer + healthcheck: + test: ["CMD-SHELL", 'airflow jobs check --job-type TriggererJob --hostname "$${HOSTNAME}"'] + interval: 30s + timeout: 10s + retries: 5 + start_period: 30s + restart: always + depends_on: + <<: *airflow-common-depends-on + airflow-init: + condition: service_completed_successfully + + airflow-init: + <<: *airflow-common + entrypoint: /bin/bash + # yamllint disable rule:line-length + command: + - -c + - | + function ver() { + printf "%04d%04d%04d%04d" $${1//./ } + } + airflow_version=$$(AIRFLOW__LOGGING__LOGGING_LEVEL=INFO && gosu airflow airflow version) + airflow_version_comparable=$$(ver $${airflow_version}) + min_airflow_version=2.2.0 + min_airflow_version_comparable=$$(ver $${min_airflow_version}) + if (( airflow_version_comparable < min_airflow_version_comparable )); then + echo + echo -e "\033[1;31mERROR!!!: Too old Airflow version $${airflow_version}!\e[0m" + echo "The minimum Airflow version supported: $${min_airflow_version}. Only use this or higher!" + echo + exit 1 + fi + if [[ -z "${AIRFLOW_UID}" ]]; then + echo + echo -e "\033[1;33mWARNING!!!: AIRFLOW_UID not set!\e[0m" + echo "If you are on Linux, you SHOULD follow the instructions below to set " + echo "AIRFLOW_UID environment variable, otherwise files will be owned by root." + echo "For other operating systems you can get rid of the warning with manually created .env file:" + echo " See: https://airflow.apache.org/docs/apache-airflow/stable/howto/docker-compose/index.html#setting-the-right-airflow-user" + echo + fi + one_meg=1048576 + mem_available=$$(($$(getconf _PHYS_PAGES) * $$(getconf PAGE_SIZE) / one_meg)) + cpus_available=$$(grep -cE 'cpu[0-9]+' /proc/stat) + disk_available=$$(df / | tail -1 | awk '{print $$4}') + warning_resources="false" + if (( mem_available < 4000 )) ; then + echo + echo -e "\033[1;33mWARNING!!!: Not enough memory available for Docker.\e[0m" + echo "At least 4GB of memory required. You have $$(numfmt --to iec $$((mem_available * one_meg)))" + echo + warning_resources="true" + fi + if (( cpus_available < 2 )); then + echo + echo -e "\033[1;33mWARNING!!!: Not enough CPUS available for Docker.\e[0m" + echo "At least 2 CPUs recommended. You have $${cpus_available}" + echo + warning_resources="true" + fi + if (( disk_available < one_meg * 10 )); then + echo + echo -e "\033[1;33mWARNING!!!: Not enough Disk space available for Docker.\e[0m" + echo "At least 10 GBs recommended. You have $$(numfmt --to iec $$((disk_available * 1024 )))" + echo + warning_resources="true" + fi + if [[ $${warning_resources} == "true" ]]; then + echo + echo -e "\033[1;33mWARNING!!!: You have not enough resources to run Airflow (see above)!\e[0m" + echo "Please follow the instructions to increase amount of resources available:" + echo " https://airflow.apache.org/docs/apache-airflow/stable/howto/docker-compose/index.html#before-you-begin" + echo + fi + exec /entrypoint airflow version + # yamllint enable rule:line-length + environment: + <<: *airflow-common-env + _AIRFLOW_DB_UPGRADE: 'true' + _AIRFLOW_WWW_USER_CREATE: 'true' + _AIRFLOW_WWW_USER_USERNAME: ${_AIRFLOW_WWW_USER_USERNAME:-airflow} + _AIRFLOW_WWW_USER_PASSWORD: ${_AIRFLOW_WWW_USER_PASSWORD:-airflow} + _PIP_ADDITIONAL_REQUIREMENTS: '' + user: "0:0" + volumes: + - ${AIRFLOW_PROJ_DIR:-.}:/sources + + airflow-cli: + <<: *airflow-common + profiles: + - debug + environment: + <<: *airflow-common-env + CONNECTION_CHECK_MAX_COUNT: "0" + # Workaround for entrypoint issue. See: https://github.com/apache/airflow/issues/16252 + command: + - bash + - -c + - airflow + + # You can enable flower by adding "--profile flower" option e.g. docker-compose --profile flower up + # or by explicitly targeted on the command line e.g. docker-compose up flower. + # See: https://docs.docker.com/compose/profiles/ + flower: + <<: *airflow-common + command: celery flower + profiles: + - flower + ports: + - "5555:5555" + healthcheck: + test: ["CMD", "curl", "--fail", "http://localhost:5555/"] + interval: 30s + timeout: 10s + retries: 5 + start_period: 30s + restart: always + depends_on: + <<: *airflow-common-depends-on + airflow-init: + condition: service_completed_successfully + +volumes: + postgres-db-volume: diff --git a/inlong-manager/manager-schedule/src/test/resources/airflow/testGroup_cron.py b/inlong-manager/manager-schedule/src/test/resources/airflow/testGroup_cron.py new file mode 100644 index 00000000000..b753eb7587c --- /dev/null +++ b/inlong-manager/manager-schedule/src/test/resources/airflow/testGroup_cron.py @@ -0,0 +1,112 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow import DAG +from datetime import datetime, timedelta +from airflow.operators.python_operator import PythonOperator +from datetime import datetime +from croniter import croniter +from airflow.hooks.base_hook import BaseHook +import requests +import pytz + +timezone = "Asia/Shanghai" +start_offset_datetime_str = 1731072908243 +end_offset_datetime_str = 1731142800000 +schedule_interval = "*/1 * * * *" # Or put cron expression +dag_id = "inlong_offline_task_testGroup_cron" +groupId = "test_offline_1" +connectionId = "inlong_connection" +boundaryType = str("time") + +target_timezone = pytz.timezone(timezone) + +start_date = datetime.fromtimestamp(start_offset_datetime_str / 1000, tz=target_timezone) +end_date = datetime.fromtimestamp(end_offset_datetime_str / 1000, tz=target_timezone) + + +def taskFunction(**context): + print("#########################") + conn = BaseHook.get_connection(connectionId) + url = f"http://{conn.host}:{conn.port}/{conn.schema}" + params = { + "username": conn.login, + "password": conn.password + } + print("params", params) + headers = { + "User-Agent": "Mozilla/5.0 (Windows NT 10.0; Win64; x64; rv:131.0) Gecko/20100101 Firefox/131.0", + "Accept": "application/json", + "Accept-Language": "zh-CN,zh;q=0.8,zh-TW;q=0.7,zh-HK;q=0.5,en-US;q=0.3,en;q=0.2", + "Accept-Encoding": "gzip, deflate", + "Referer": "http://192.168.101.2:8083/", + "Content-Type": "application/json;charset=UTF-8", + "tenant": "public", + "Origin": "http://192.168.101.2", + "Connection": "close", + "Priority": "u=0" + } + time_interval = get_time_interval(context) + data = { + "boundaryType": boundaryType, + "groupId": groupId, + "lowerBoundary": time_interval[0], + "upperBoundary": time_interval[1] + } + print("Request Body: ", data) + response = requests.post(url, params=params, headers=headers, json=data) + if response.status_code == 200: + print(response.json()) + else: + print(response.text) + print("#########################") + + +def get_time_interval(context): + execution_date = context.get('execution_date') + execution_date = execution_date.astimezone(target_timezone) + dag = context.get('dag') + schedule_interval = dag.schedule_interval + if isinstance(schedule_interval, timedelta): + return execution_date.timestamp(), (execution_date + schedule_interval).timestamp() + else: + cron_expr = dag.schedule_interval + cron = croniter(cron_expr, execution_date) + next_run = cron.get_next(datetime) + return execution_date.timestamp(), next_run.timestamp() + + +default_args = { + 'owner': 'inlong', + 'start_date': start_date, + 'end_date': end_date, + 'catchup': False, +} + +dag = DAG( + dag_id, + default_args=default_args, + schedule_interval=schedule_interval, + is_paused_upon_creation=False +) + +clean_task = PythonOperator( + task_id=dag_id, + python_callable=taskFunction, + provide_context=True, + dag=dag, +) diff --git a/inlong-manager/manager-schedule/src/test/resources/airflow/testGroup_normal.py b/inlong-manager/manager-schedule/src/test/resources/airflow/testGroup_normal.py new file mode 100644 index 00000000000..5666f9f471a --- /dev/null +++ b/inlong-manager/manager-schedule/src/test/resources/airflow/testGroup_normal.py @@ -0,0 +1,110 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from airflow import DAG +from datetime import datetime, timedelta +from airflow.operators.python_operator import PythonOperator +from datetime import datetime +from croniter import croniter +from airflow.hooks.base_hook import BaseHook +import requests +import pytz + +timezone = "Asia/Shanghai" +start_offset_datetime_str = 1731072908243 +end_offset_datetime_str = 1731142800000 +schedule_interval = "*/1 * * * *" # Or put cron expression +dag_id = "inlong_offline_task_testGroup_normal" +groupId = "test_offline_1" +connectionId = "inlong_connection" +boundaryType = str("time") + +target_timezone = pytz.timezone(timezone) + +start_date = datetime.fromtimestamp(start_offset_datetime_str / 1000, tz=target_timezone) +end_date = datetime.fromtimestamp(end_offset_datetime_str / 1000, tz=target_timezone) + + +def taskFunction(**context): + print("#########################") + conn = BaseHook.get_connection(connectionId) + url = f"http://{conn.host}:{conn.port}/{conn.schema}" + params = { + "username": conn.login, + "password": conn.password + } + print("params", params) + headers = { + "User-Agent": "Mozilla/5.0 (Windows NT 10.0; Win64; x64; rv:131.0) Gecko/20100101 Firefox/131.0", + "Accept": "application/json", + "Accept-Language": "zh-CN,zh;q=0.8,zh-TW;q=0.7,zh-HK;q=0.5,en-US;q=0.3,en;q=0.2", + "Accept-Encoding": "gzip, deflate", + "Content-Type": "application/json;charset=UTF-8", + "tenant": "public", + "Connection": "close", + "Priority": "u=0" + } + time_interval = get_time_interval(context) + data = { + "boundaryType": boundaryType, + "groupId": groupId, + "lowerBoundary": time_interval[0], + "upperBoundary": time_interval[1] + } + print("Request Body: ", data) + response = requests.post(url, params=params, headers=headers, json=data) + if response.status_code == 200: + print(response.json()) + else: + print(response.text) + print("#########################") + + +def get_time_interval(context): + execution_date = context.get('execution_date') + execution_date = execution_date.astimezone(target_timezone) + dag = context.get('dag') + schedule_interval = dag.schedule_interval + if isinstance(schedule_interval, timedelta): + return execution_date.timestamp(), (execution_date + schedule_interval).timestamp() + else: + cron_expr = dag.schedule_interval + cron = croniter(cron_expr, execution_date) + next_run = cron.get_next(datetime) + return execution_date.timestamp(), next_run.timestamp() + + +default_args = { + 'owner': 'inlong', + 'start_date': start_date, + 'end_date': end_date, + 'catchup': False, +} + +dag = DAG( + dag_id, + default_args=default_args, + schedule_interval=schedule_interval, + is_paused_upon_creation=False +) + +clean_task = PythonOperator( + task_id=dag_id, + python_callable=taskFunction, + provide_context=True, + dag=dag, +) diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/cluster/InlongClusterServiceImpl.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/cluster/InlongClusterServiceImpl.java index e2be48a4e02..a4b65e6c2ec 100644 --- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/cluster/InlongClusterServiceImpl.java +++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/cluster/InlongClusterServiceImpl.java @@ -960,7 +960,7 @@ public Boolean testSSHConnection(ClusterNodeRequest request) { public DataProxyNodeResponse getDataProxyNodes(String groupId, String protocolType) { LOGGER.debug("begin to get data proxy nodes for groupId={}, protocol={}", groupId, protocolType); - InlongGroupEntity groupEntity = groupMapper.selectByGroupId(groupId); + InlongGroupEntity groupEntity = groupMapper.selectByGroupIdWithoutTenant(groupId); if (groupEntity == null) { String errMsg = String.format("group not found by groupId=%s", groupId); LOGGER.error(errMsg); @@ -1082,7 +1082,7 @@ public DataProxyNodeResponse getDataProxyNodesByCluster(String clusterName, Stri } private List getClusterNodes(String groupId, String clusterType, String protocolType) { - InlongGroupEntity groupEntity = groupMapper.selectByGroupId(groupId); + InlongGroupEntity groupEntity = groupMapper.selectByGroupIdWithoutTenant(groupId); if (groupEntity == null) { LOGGER.warn("inlong group not exists for groupId={}", groupId); return Lists.newArrayList(); diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/dirtyData/DirtyQueryLogService.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/dirtyData/DirtyQueryLogService.java new file mode 100644 index 00000000000..3629ee985c7 --- /dev/null +++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/dirtyData/DirtyQueryLogService.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.service.dirtyData; + +import org.apache.inlong.manager.pojo.sink.DirtyDataDetailResponse; +import org.apache.inlong.manager.pojo.sink.DirtyDataRequest; +import org.apache.inlong.manager.pojo.sink.DirtyDataResponse; +import org.apache.inlong.manager.pojo.sink.DirtyDataTrendDetailResponse; +import org.apache.inlong.manager.pojo.sink.DirtyDataTrendRequest; + +import java.util.List; + +/** + * Dirty query log service + */ +public interface DirtyQueryLogService { + + DirtyDataResponse listDirtyData(DirtyDataRequest request); + + DirtyDataResponse listDirtyDataTrend(DirtyDataTrendRequest request); + + List getDirtyData(String taskId); + + List getDirtyDataTrend(String taskId); + + String getSqlTaskStatus(String taskId); + +} diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/dirtyData/impl/DirtyQueryLogServiceImpl.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/dirtyData/impl/DirtyQueryLogServiceImpl.java new file mode 100644 index 00000000000..3067ddecc95 --- /dev/null +++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/dirtyData/impl/DirtyQueryLogServiceImpl.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.manager.service.dirtyData.impl; + +import org.apache.inlong.manager.common.exceptions.BusinessException; +import org.apache.inlong.manager.dao.entity.DirtyQueryLogEntity; +import org.apache.inlong.manager.dao.mapper.DirtyQueryLogEntityMapper; +import org.apache.inlong.manager.pojo.sink.DirtyDataDetailResponse; +import org.apache.inlong.manager.pojo.sink.DirtyDataRequest; +import org.apache.inlong.manager.pojo.sink.DirtyDataResponse; +import org.apache.inlong.manager.pojo.sink.DirtyDataTrendDetailResponse; +import org.apache.inlong.manager.pojo.sink.DirtyDataTrendRequest; +import org.apache.inlong.manager.pojo.user.LoginUserUtils; +import org.apache.inlong.manager.service.dirtyData.DirtyQueryLogService; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.commons.collections.CollectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.stereotype.Service; + +import javax.annotation.PostConstruct; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; + +@Service +@Slf4j +public class DirtyQueryLogServiceImpl implements DirtyQueryLogService { + + private static final Logger LOGGER = LoggerFactory.getLogger(DirtyQueryLogServiceImpl.class); + + @Autowired + private DirtyQueryLogEntityMapper dirtyQueryLogEntityMapper; + @Autowired + private ObjectMapper objectMapper; + + @Value("${dirty.log.clean.enabled:false}") + private Boolean dirtyLogCleanEnabled; + @Value("${dirty.log.clean.interval.minutes:5}") + private Integer dirtyLogCleanInterval; + @Value("${dirty.log.retention.minutes:10}") + private Integer retentionMinutes; + @Value("${dirty.log.db.table:inlong_iceberg::dirty_data_achive_iceberg}") + private String dirtyDataDbTable; + + @PostConstruct + private void startDirtyLogCleanTask() { + if (dirtyLogCleanEnabled) { + ThreadFactory factory = new ThreadFactoryBuilder() + .setNameFormat("scheduled-dirtyQueryLog-deleted-%d") + .setDaemon(true) + .build(); + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(factory); + executor.scheduleWithFixedDelay(() -> { + try { + LOGGER.info("begin to clean dirty query log"); + dirtyQueryLogEntityMapper.updateToTimeout(retentionMinutes); + LOGGER.info("success to clean dirty query log successfully"); + } catch (Throwable t) { + LOGGER.error("clean dirty query log error", t); + } + }, 0, dirtyLogCleanInterval, TimeUnit.MINUTES); + LOGGER.info("clean dirty query log started successfully"); + } + } + + @Override + public DirtyDataResponse listDirtyDataTrend(DirtyDataTrendRequest request) { + if (CollectionUtils.isEmpty(request.getSinkIdList())) { + return null; + } + try { + DirtyDataResponse dirtyDataResponse = new DirtyDataResponse(); + String requestStr = objectMapper.writeValueAsString(request); + String md5 = DigestUtils.md5Hex(requestStr); + DirtyQueryLogEntity dirtyQueryLogEntity = dirtyQueryLogEntityMapper.selectByMd5(md5); + if (dirtyQueryLogEntity != null) { + LOGGER.info("dirty query log is exist"); + dirtyDataResponse.setTaskId(dirtyQueryLogEntity.getTaskId()); + return dirtyDataResponse; + } + DirtyQueryLogEntity dirtyQueryLog = new DirtyQueryLogEntity(); + // TODO dirtyQueryLog.setTaskId(); + dirtyQueryLog.setMd5(md5); + dirtyQueryLog.setRequestParams(requestStr); + dirtyQueryLog.setCreator(LoginUserUtils.getLoginUser().getName()); + dirtyQueryLog.setModifier(LoginUserUtils.getLoginUser().getName()); + dirtyQueryLogEntityMapper.insert(dirtyQueryLog); + + return dirtyDataResponse; + } catch (Exception e) { + throw new BusinessException("list dirty data trend failed"); + } + } + + @Override + public DirtyDataResponse listDirtyData(DirtyDataRequest request) { + if (CollectionUtils.isEmpty(request.getSinkIdList())) { + return null; + } + if (request.getDataCount() == null) { + request.setDataCount(10); + } + try { + DirtyDataResponse dirtyDataResponse = new DirtyDataResponse(); + String requestStr = objectMapper.writeValueAsString(request); + String md5 = DigestUtils.md5Hex(requestStr); + DirtyQueryLogEntity dirtyQueryLogEntity = dirtyQueryLogEntityMapper.selectByMd5(md5); + if (dirtyQueryLogEntity != null) { + LOGGER.info("dirty query log is exist"); + dirtyDataResponse.setTaskId(dirtyQueryLogEntity.getTaskId()); + return dirtyDataResponse; + } + DirtyQueryLogEntity dirtyQueryLog = new DirtyQueryLogEntity(); + // TODO dirtyQueryLog.setTaskId(); + dirtyQueryLog.setMd5(md5); + dirtyQueryLog.setRequestParams(requestStr); + dirtyQueryLog.setCreator(LoginUserUtils.getLoginUser().getName()); + dirtyQueryLog.setModifier(LoginUserUtils.getLoginUser().getName()); + dirtyQueryLogEntityMapper.insert(dirtyQueryLog); + return dirtyDataResponse; + } catch (Exception e) { + LOGGER.error("list dirty data failed", e); + throw new BusinessException("list dirty data failed"); + } + } + + @Override + public List getDirtyData(String taskId) { + try { + // TODO + return new ArrayList<>(); + } catch (Exception e) { + LOGGER.error("get dirty data failed", e); + throw new BusinessException("get dirty data failed"); + } + } + + @Override + public List getDirtyDataTrend(String taskId) { + try { + // TODO + return new ArrayList<>(); + } catch (Exception e) { + LOGGER.error("get dirty data trend failed", e); + throw new BusinessException("get dirty data trend failed"); + } + } + + @Override + public String getSqlTaskStatus(String taskId) { + try { + // TODO + return "success"; + } catch (Exception e) { + LOGGER.error("get sql task status failed", e); + throw new BusinessException("get get sql task status failed"); + } + } +} diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/pulsar/PulsarQueueResourceOperator.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/pulsar/PulsarQueueResourceOperator.java index efbdeabf206..67a1b39f7a5 100644 --- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/pulsar/PulsarQueueResourceOperator.java +++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/resource/queue/pulsar/PulsarQueueResourceOperator.java @@ -324,7 +324,7 @@ public List queryLatestMessages(InlongGroupInfo groupInfo, Inlon int finalMsgCount = Math.min(request.getMessageCount(), briefMQMessages.size()); if (finalMsgCount > 0) { - return briefMQMessages.subList(0, finalMsgCount); + return new ArrayList<>(briefMQMessages.subList(0, finalMsgCount)); } else { return new ArrayList<>(); } diff --git a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/schedule/ScheduleOperatorImpl.java b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/schedule/ScheduleOperatorImpl.java index 6ef899ab9e3..c00bb24fc4f 100644 --- a/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/schedule/ScheduleOperatorImpl.java +++ b/inlong-manager/manager-service/src/main/java/org/apache/inlong/manager/service/schedule/ScheduleOperatorImpl.java @@ -59,8 +59,6 @@ public class ScheduleOperatorImpl implements ScheduleOperator { private OfflineJobOperator offlineJobOperator; - private ScheduleEngineClient scheduleEngineClient; - @Override @Transactional(rollbackFor = Throwable.class) public int saveOpt(ScheduleInfoRequest request, String operator) { @@ -89,10 +87,7 @@ private void registerScheduleInfoForApprovedGroup(ScheduleInfo scheduleInfo, Str } private ScheduleEngineClient getScheduleEngineClient(String scheduleEngine) { - if (scheduleEngineClient == null) { - scheduleEngineClient = scheduleClientFactory.getInstance(scheduleEngine); - } - return scheduleEngineClient; + return scheduleClientFactory.getInstance(scheduleEngine); } @Override diff --git a/inlong-manager/manager-test/src/main/resources/h2/apache_inlong_manager.sql b/inlong-manager/manager-test/src/main/resources/h2/apache_inlong_manager.sql index 1d92f14f468..6836d44d5b5 100644 --- a/inlong-manager/manager-test/src/main/resources/h2/apache_inlong_manager.sql +++ b/inlong-manager/manager-test/src/main/resources/h2/apache_inlong_manager.sql @@ -1013,6 +1013,24 @@ CREATE TABLE IF NOT EXISTS `schedule_config` UNIQUE KEY `unique_group_schedule_config` (`inlong_group_id`, `is_deleted`) ) ENGINE = InnoDB DEFAULT CHARSET = utf8mb4 COMMENT = 'schedule_config'; + +-- ---------------------------- +-- Table structure for dirty_query_log +-- ---------------------------- +CREATE TABLE IF NOT EXISTS `dirty_query_log` +( + `id` int(11) NOT NULL AUTO_INCREMENT COMMENT 'Incremental primary key', + `md5` varchar(256) NOT NULL COMMENT 'Md5 for request params', + `request_params` mediumtext DEFAULT NULL COMMENT 'Request params, will be saved as JSON string', + `task_id` varchar(256) DEFAULT '' COMMENT 'Task id', + `is_deleted` int(11) DEFAULT '0' COMMENT 'Whether to delete, 0: not deleted, > 0: deleted', + `creator` varchar(64) NOT NULL COMMENT 'Creator name', + `modifier` varchar(64) DEFAULT NULL COMMENT 'Modifier name', + `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'Create time', + `modify_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP COMMENT 'Modify time', + `version` int(11) NOT NULL DEFAULT '1' COMMENT 'Version number, which will be incremented by 1 after modification', + PRIMARY KEY (`id`) +); -- ---------------------------- diff --git a/inlong-manager/manager-web/sql/apache_inlong_manager.sql b/inlong-manager/manager-web/sql/apache_inlong_manager.sql index 58259c90477..51a72d60399 100644 --- a/inlong-manager/manager-web/sql/apache_inlong_manager.sql +++ b/inlong-manager/manager-web/sql/apache_inlong_manager.sql @@ -1066,6 +1066,25 @@ CREATE TABLE IF NOT EXISTS `schedule_config` UNIQUE KEY `unique_group_schedule_config` (`inlong_group_id`, `is_deleted`) ) ENGINE = InnoDB DEFAULT CHARSET = utf8mb4 COMMENT = 'schedule_config'; + +-- ---------------------------- +-- Table structure for dirty_query_log +-- ---------------------------- +CREATE TABLE IF NOT EXISTS `dirty_query_log` +( + `id` int(11) NOT NULL AUTO_INCREMENT COMMENT 'Incremental primary key', + `md5` varchar(256) NOT NULL COMMENT 'Md5 for request params', + `request_params` mediumtext DEFAULT NULL COMMENT 'Request params, will be saved as JSON string', + `task_id` varchar(256) DEFAULT '' COMMENT 'Task id', + `is_deleted` int(11) DEFAULT '0' COMMENT 'Whether to delete, 0: not deleted, > 0: deleted', + `creator` varchar(64) NOT NULL COMMENT 'Creator name', + `modifier` varchar(64) DEFAULT NULL COMMENT 'Modifier name', + `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'Create time', + `modify_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP COMMENT 'Modify time', + `version` int(11) NOT NULL DEFAULT '1' COMMENT 'Version number, which will be incremented by 1 after modification', + PRIMARY KEY (`id`) +) ENGINE = InnoDB + DEFAULT CHARSET = utf8mb4 COMMENT ='Dirty query log table'; -- ---------------------------- SET FOREIGN_KEY_CHECKS = 1; diff --git a/inlong-manager/manager-web/sql/changes-2.1.0.sql b/inlong-manager/manager-web/sql/changes-2.1.0.sql index 0d4c9847781..2fa48f95b45 100644 --- a/inlong-manager/manager-web/sql/changes-2.1.0.sql +++ b/inlong-manager/manager-web/sql/changes-2.1.0.sql @@ -25,3 +25,19 @@ USE `apache_inlong_manager`; ALTER TABLE `schedule_config` ADD COLUMN `schedule_engine` varchar(64) NOT NULL DEFAULT 'Quartz' COMMENT 'Schedule engine, support Quartz, Airflow and DolphinScheduler'; + +CREATE TABLE IF NOT EXISTS `dirty_query_log` +( + `id` int(11) NOT NULL AUTO_INCREMENT COMMENT 'Incremental primary key', + `md5` varchar(256) NOT NULL COMMENT 'Md5 for request params', + `request_params` mediumtext DEFAULT NULL COMMENT 'Request params, will be saved as JSON string', + `task_id` varchar(256) DEFAULT '' COMMENT 'Task id', + `is_deleted` int(11) DEFAULT '0' COMMENT 'Whether to delete, 0: not deleted, > 0: deleted', + `creator` varchar(64) NOT NULL COMMENT 'Creator name', + `modifier` varchar(64) DEFAULT NULL COMMENT 'Modifier name', + `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'Create time', + `modify_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP COMMENT 'Modify time', + `version` int(11) NOT NULL DEFAULT '1' COMMENT 'Version number, which will be incremented by 1 after modification', + PRIMARY KEY (`id`) +) ENGINE = InnoDB + DEFAULT CHARSET = utf8mb4 COMMENT ='Dirty query log table'; diff --git a/inlong-manager/manager-web/src/main/java/org/apache/inlong/manager/web/controller/StreamSinkController.java b/inlong-manager/manager-web/src/main/java/org/apache/inlong/manager/web/controller/StreamSinkController.java index b7b8dbd5d40..331e3a53553 100644 --- a/inlong-manager/manager-web/src/main/java/org/apache/inlong/manager/web/controller/StreamSinkController.java +++ b/inlong-manager/manager-web/src/main/java/org/apache/inlong/manager/web/controller/StreamSinkController.java @@ -25,12 +25,18 @@ import org.apache.inlong.manager.pojo.common.PageResult; import org.apache.inlong.manager.pojo.common.Response; import org.apache.inlong.manager.pojo.common.UpdateResult; +import org.apache.inlong.manager.pojo.sink.DirtyDataDetailResponse; +import org.apache.inlong.manager.pojo.sink.DirtyDataRequest; +import org.apache.inlong.manager.pojo.sink.DirtyDataResponse; +import org.apache.inlong.manager.pojo.sink.DirtyDataTrendDetailResponse; +import org.apache.inlong.manager.pojo.sink.DirtyDataTrendRequest; import org.apache.inlong.manager.pojo.sink.ParseFieldRequest; import org.apache.inlong.manager.pojo.sink.SinkField; import org.apache.inlong.manager.pojo.sink.SinkPageRequest; import org.apache.inlong.manager.pojo.sink.SinkRequest; import org.apache.inlong.manager.pojo.sink.StreamSink; import org.apache.inlong.manager.pojo.user.LoginUserUtils; +import org.apache.inlong.manager.service.dirtyData.DirtyQueryLogService; import org.apache.inlong.manager.service.operationlog.OperationLog; import org.apache.inlong.manager.service.sink.StreamSinkService; @@ -60,6 +66,8 @@ public class StreamSinkController { @Autowired private StreamSinkService sinkService; + @Autowired + private DirtyQueryLogService dirtyQueryLogService; @RequestMapping(value = "/sink/save", method = RequestMethod.POST) @OperationLog(operation = OperationType.CREATE, operationTarget = OperationTarget.SINK) @@ -143,4 +151,34 @@ public Response> parseFields(@RequestBody ParseFieldRequest pars return Response.success(sinkService.parseFields(parseFieldRequest)); } + @RequestMapping(value = "/sink/listDirtyData", method = RequestMethod.POST) + @ApiOperation(value = "List stream sinks by paginating") + public Response listDirtyData(@RequestBody DirtyDataRequest request) { + return Response.success(dirtyQueryLogService.listDirtyData(request)); + } + + @RequestMapping(value = "/sink/listDirtyDataTrend", method = RequestMethod.POST) + @ApiOperation(value = "List stream sinks by paginating") + public Response listDirtyDataTrend(@RequestBody DirtyDataTrendRequest request) { + return Response.success(dirtyQueryLogService.listDirtyDataTrend(request)); + } + + @RequestMapping(value = "/sink/getDirtyData/{taskId}", method = RequestMethod.GET) + @ApiImplicitParam(name = "taskId", dataTypeClass = String.class, required = true) + public Response> getDirtyData(@PathVariable String taskId) { + return Response.success(dirtyQueryLogService.getDirtyData(taskId)); + } + + @RequestMapping(value = "/sink/getDirtyDataTrend/{taskId}", method = RequestMethod.GET) + @ApiImplicitParam(name = "taskId", dataTypeClass = String.class, required = true) + public Response> getDirtyDataTrend(@PathVariable String taskId) { + return Response.success(dirtyQueryLogService.getDirtyDataTrend(taskId)); + } + + @RequestMapping(value = "/sink/SqlTaskStatus/{taskId}", method = RequestMethod.GET) + @ApiImplicitParam(name = "taskId", dataTypeClass = String.class, required = true) + public Response SqlTaskStatus(@PathVariable String taskId) { + return Response.success(dirtyQueryLogService.getSqlTaskStatus(taskId)); + } + } diff --git a/inlong-manager/manager-web/src/main/resources/application-dev.properties b/inlong-manager/manager-web/src/main/resources/application-dev.properties index 794f201bda7..8a9ae13ccca 100644 --- a/inlong-manager/manager-web/src/main/resources/application-dev.properties +++ b/inlong-manager/manager-web/src/main/resources/application-dev.properties @@ -103,6 +103,24 @@ agent.install.temp.path=inlong/agent-installer-temp/ # The primary key id of the default agent module used default.module.id=1 -# schedule engine type -# support none(no scheduler) and quartz(quartz scheduler), default is none -inlong.schedule.engine=none \ No newline at end of file + +# Dirty log +dirty.log.clean.enabled=false +dirty.log.clean.interval.minutes=5 +dirty.log.retention.minutes=10 +dirty.log.db.table=inlong_iceberg::dirty_data_achive_iceberg + +# Inlong Manager URL accessible by the scheduler +schedule.engine.inlong.manager.url=http://127.0.0.1:8083 + +# DolphinScheduler related config +schedule.engine.dolphinscheduler.url= +schedule.engine.dolphinscheduler.token= + +# Airflow related config +schedule.engine.airflow.baseUrl= +schedule.engine.airflow.username= +schedule.engine.airflow.password= +schedule.engine.airflow.connection.id= +schedule.engine.airflow.cleaner.id= +schedule.engine.airflow.creator.id= \ No newline at end of file diff --git a/inlong-manager/manager-web/src/main/resources/application-prod.properties b/inlong-manager/manager-web/src/main/resources/application-prod.properties index 3e8f329470a..f5bde10caf2 100644 --- a/inlong-manager/manager-web/src/main/resources/application-prod.properties +++ b/inlong-manager/manager-web/src/main/resources/application-prod.properties @@ -95,6 +95,23 @@ group.deleted.enabled=false # Tencent cloud log service endpoint, The Operator cls resource by it cls.manager.endpoint=127.0.0.1 -# schedule engine type -# support none(no scheduler) and quartz(quartz scheduler), default is none -inlong.schedule.engine=none \ No newline at end of file +# Dirty log +dirty.log.clean.enabled=false +dirty.log.clean.interval.minutes=5 +dirty.log.retention.minutes=10 +dirty.log.db.table=inlong_iceberg::dirty_data_achive_iceberg + +# Inlong Manager URL accessible by the scheduler +schedule.engine.inlong.manager.url=http://127.0.0.1:8083 + +# DolphinScheduler related config +schedule.engine.dolphinscheduler.url= +schedule.engine.dolphinscheduler.token= + +# Airflow related config +schedule.engine.airflow.baseUrl= +schedule.engine.airflow.username= +schedule.engine.airflow.password= +schedule.engine.airflow.connection.id= +schedule.engine.airflow.cleaner.id= +schedule.engine.airflow.creator.id= \ No newline at end of file diff --git a/inlong-manager/manager-web/src/main/resources/application-test.properties b/inlong-manager/manager-web/src/main/resources/application-test.properties index 5ff929c2b82..905915df6b9 100644 --- a/inlong-manager/manager-web/src/main/resources/application-test.properties +++ b/inlong-manager/manager-web/src/main/resources/application-test.properties @@ -96,6 +96,23 @@ group.deleted.enabled=false # Tencent cloud log service endpoint, The Operator cls resource by it cls.manager.endpoint=127.0.0.1 -# schedule engine type -# support none(no scheduler) and quartz(quartz scheduler), default is none -inlong.schedule.engine=none \ No newline at end of file +# Dirty log +dirty.log.clean.enabled=false +dirty.log.clean.interval.minutes=5 +dirty.log.retention.minutes=10 +dirty.log.db.table=inlong_iceberg::dirty_data_achive_iceberg + +# Inlong Manager URL accessible by the scheduler +schedule.engine.inlong.manager.url=http://127.0.0.1:8083 + +# DolphinScheduler related config +schedule.engine.dolphinscheduler.url= +schedule.engine.dolphinscheduler.token= + +# Airflow related config +schedule.engine.airflow.baseUrl= +schedule.engine.airflow.username= +schedule.engine.airflow.password= +schedule.engine.airflow.connection.id= +schedule.engine.airflow.cleaner.id= +schedule.engine.airflow.creator.id= \ No newline at end of file diff --git a/inlong-sdk/dataproxy-sdk-twins/dataproxy-sdk-golang/connpool/connpool.go b/inlong-sdk/dataproxy-sdk-twins/dataproxy-sdk-golang/connpool/connpool.go index 0c5b215fff9..d8f3a1a3baa 100755 --- a/inlong-sdk/dataproxy-sdk-twins/dataproxy-sdk-golang/connpool/connpool.go +++ b/inlong-sdk/dataproxy-sdk-twins/dataproxy-sdk-golang/connpool/connpool.go @@ -266,7 +266,7 @@ func (p *connPool) put(conn gnet.Conn, err error, isNewConn bool) { addr := remoteAddr.String() if _, ok := p.endpointMap.Load(addr); !ok { - p.log.Info("endpoint deleted, close its connection, addr:", addr) + p.log.Warn("endpoint deleted, close its connection, addr:", addr) CloseConn(conn, defaultConnCloseDelay) return } @@ -348,7 +348,7 @@ func (p *connPool) UpdateEndpoints(all, add, del []string) { if len(delEndpoints) > 0 { // delete connections for deleted endpoints - p.log.Info("delete old connections...") + p.log.Debug("delete old connections...") // use a temp slice to store the conn in connChan tempConns := make([]gnet.Conn, 0, cap(p.connChan)) @@ -365,7 +365,7 @@ func (p *connPool) UpdateEndpoints(all, add, del []string) { addr := remoteAddr.String() if _, ok := delEndpoints[addr]; ok { - p.log.Info("endpoint deleted, close its connection, addr:", addr) + p.log.Warn("endpoint deleted, close its connection, addr:", addr) CloseConn(conn, defaultConnCloseDelay) // for the deleted endpoint, we decrease its conn count before it is really closed, so that we can avoid creating more conns than we expect when rebalance p.decEndpointConnCount(addr) @@ -464,7 +464,7 @@ loop: } func (p *connPool) markUnavailable(ep string) { - p.log.Info("endpoint cannot be connected, marking as unavailable, addr: ", ep) + p.log.Debug("endpoint cannot be connected, marking as unavailable, addr: ", ep) p.unavailable.Store(ep, time.Now()) p.retryCounts.Store(ep, 0) } @@ -500,27 +500,27 @@ func (p *connPool) recoverAndRebalance() { } func (p *connPool) dump() { - p.log.Info("all endpoints:") + p.log.Debug("all endpoints:") eps := p.endpoints.Load() endpoints, ok := eps.([]string) if ok { for _, ep := range endpoints { - p.log.Info(ep) + p.log.Debug(ep) } } dump := false p.unavailable.Range(func(key, value any) bool { if !dump { - p.log.Info("unavailable endpoints:") + p.log.Debug("unavailable endpoints:") } - p.log.Info(key) + p.log.Debug(key) return true }) - p.log.Info("opened connections:") + p.log.Debug("opened connections:") p.endpointConnCounts.Range(func(key, value any) bool { - p.log.Info("endpoint: ", key, ", conns: ", value.(int)) + p.log.Debug("endpoint: ", key, ", conns: ", value.(int)) return true }) } @@ -537,7 +537,7 @@ func (p *connPool) recover() bool { // try to create new conn conn, err := p.dialer.Dial(key.(string)) if err == nil { - p.log.Info("endpoint recovered, addr: ", key) + p.log.Debug("endpoint recovered, addr: ", key) p.put(conn, nil, true) p.unavailable.Delete(key) p.retryCounts.Delete(key) @@ -552,7 +552,7 @@ func (p *connPool) recover() bool { return true }) if recovered { - p.log.Info("recover triggered") + p.log.Debug("recover triggered") } return recovered } @@ -586,40 +586,40 @@ func (p *connPool) getAvailableEndpointCount() int { func (p *connPool) getExpectedConnPerEndpoint() int { // current conn count, 'cause our conn is delayed closed, curConnCount may include the ones are being closing, and basically bigger than p.requiredConnNum curConnCount := p.getConnCount() - p.log.Info("curConnCount: ", curConnCount) + p.log.Debug("curConnCount: ", curConnCount) if curConnCount <= 0 { return 1 } // initial conn count initConnCount := float64(p.requiredConnNum) - p.log.Info("initConnCount: ", initConnCount) + p.log.Debug("initConnCount: ", initConnCount) // average conn count, as curConnCount may be not accurate, we use avgConnCount as a reference avgConnCount := (curConnCount + p.requiredConnNum) >> 1 - p.log.Info("avgConnCount: ", avgConnCount) + p.log.Debug("avgConnCount: ", avgConnCount) if avgConnCount <= 0 { return 1 } // available endpoint count availableEndpointCount := p.getAvailableEndpointCount() - p.log.Info("availableEndpointCount: ", availableEndpointCount) + p.log.Debug("availableEndpointCount: ", availableEndpointCount) if availableEndpointCount <= 0 { return 1 } // curConnCount/availableEndpointCount, estimate a new conn count per endpoint estimatedVal := math.Floor(float64(curConnCount) / float64(availableEndpointCount)) - p.log.Info("conns per endpoint by current conn count: ", estimatedVal) + p.log.Debug("conns per endpoint by current conn count: ", estimatedVal) // avgConnCount/availableEndpointCount, as a reference value averageVal := math.Floor(float64(avgConnCount) / float64(availableEndpointCount)) - p.log.Info("conns per endpoint by average conn count: ", averageVal) + p.log.Debug("conns per endpoint by average conn count: ", averageVal) // initial conn count per endpoint initialVal := float64(p.connsPerEndpoint) - p.log.Info("conns per endpoint of initialization: ", initialVal) + p.log.Debug("conns per endpoint of initialization: ", initialVal) result := averageVal // nolint:ineffassign if estimatedVal < initialVal { @@ -636,7 +636,7 @@ func (p *connPool) getExpectedConnPerEndpoint() int { // at least 1 conn result = math.Max(1, result) - p.log.Info("expectedConnPerEndpoint: ", result) + p.log.Debug("expectedConnPerEndpoint: ", result) return int(result) } @@ -660,7 +660,7 @@ func (p *connPool) rebalance() { for i := currentCount; i < expectedConnPerEndpoint; i++ { conn, err := p.dialNewConn(addr) if err == nil { - p.log.Info("adding connection for addr: ", addr) + p.log.Debug("adding connection for addr: ", addr) p.put(conn, nil, true) rebalanced = true } else { @@ -684,7 +684,7 @@ func (p *connPool) rebalance() { for i := 0; i < expectedConnPerEndpoint; i++ { conn, err := p.dialNewConn(addr) if err == nil { - p.log.Info("adding connection for addr: ", addr) + p.log.Debug("adding connection for addr: ", addr) p.put(conn, nil, true) rebalanced = true } else { @@ -696,7 +696,7 @@ func (p *connPool) rebalance() { }) if rebalanced { - p.log.Info("rebalance triggered") + p.log.Debug("rebalance triggered") } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/ConfigConstants.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/ConfigConstants.java index 26f8d131b4b..7adc5087afa 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/ConfigConstants.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/ConfigConstants.java @@ -21,8 +21,57 @@ public class ConfigConstants { public static final String PROXY_SDK_VERSION = "1.2.11"; - public static final int ALIVE_CONNECTIONS = 3; - public static final int MAX_TIMEOUT_CNT = 3; + public static String HTTP = "http://"; + public static String HTTPS = "https://"; + + // dataproxy node config + public static final String MANAGER_DATAPROXY_API = "/inlong/manager/openapi/dataproxy/getIpList/"; + public static final String META_STORE_SUB_DIR = "/.inlong/"; + public static final String LOCAL_DP_CONFIG_FILE_SUFFIX = ".local"; + public static final String REMOTE_DP_CACHE_FILE_SUFFIX = ".proxyip"; + public static final String REMOTE_ENCRYPT_CACHE_FILE_SUFFIX = ".pubKey"; + // authorization key + public static final String BASIC_AUTH_HEADER = "authorization"; + + // config info sync interval in minutes + public static final int VAL_DEF_CONFIG_SYNC_INTERVAL_MIN = 3; + public static final int VAL_MIN_CONFIG_SYNC_INTERVAL_MIN = 1; + public static final int VAL_MAX_CONFIG_SYNC_INTERVAL_MIN = 30; + public static final long VAL_UNIT_MIN_TO_MS = 60 * 1000L; + // config info sync max retry if failure + public static final int VAL_DEF_RETRY_IF_CONFIG_SYNC_FAIL = 2; + public static final int VAL_MAX_RETRY_IF_CONFIG_SYNC_FAIL = 5; + // cache config expired time in ms + public static final long VAL_DEF_CACHE_CONFIG_EXPIRED_MS = 20 * 60 * 1000L; + // node force choose interval in ms + public static final long VAL_DEF_FORCE_CHOOSE_INR_MS = 10 * 60 * 1000L; + public static final long VAL_MIN_FORCE_CHOOSE_INR_MS = 30 * 1000L; + + // connection timeout in milliseconds + public static final int VAL_DEF_CONNECT_TIMEOUT_MS = 8000; + public static final int VAL_MIN_CONNECT_TIMEOUT_MS = 2000; + public static final int VAL_MAX_CONNECT_TIMEOUT_MS = 60000; + public static final int VAL_DEF_CONNECT_CLOSE_DELAY_MS = 500; + // socket timeout in milliseconds + public static final int VAL_DEF_SOCKET_TIMEOUT_MS = 20000; + public static final int VAL_MIN_SOCKET_TIMEOUT_MS = 2000; + public static final int VAL_MAX_SOCKET_TIMEOUT_MS = 60000; + // active connects + public static final int VAL_DEF_ALIVE_CONNECTIONS = 6; + public static final int VAL_MIN_ALIVE_CONNECTIONS = 1; + // request timeout in milliseconds + public static final long VAL_DEF_REQUEST_TIMEOUT_MS = 10000L; + public static final long VAL_MIN_REQUEST_TIMEOUT_MS = 500L; + // reconnect wait ms + public static final long VAL_DEF_RECONNECT_WAIT_MS = 1000L; + public static final long VAL_MAX_RECONNECT_WAIT_MS = 180000L; + // socket buffer size + public static final int DEFAULT_SEND_BUFFER_SIZE = 16777216; + public static final int DEFAULT_RECEIVE_BUFFER_SIZE = 16777216; + // max inflight msg count per connection + public static final long MAX_INFLIGHT_MSG_COUNT_PER_CONNECTION = 10000L; + + public static final int MAX_TIMEOUT_CNT = 10; public static final int LOAD_THRESHOLD = 0; public static final int CYCLE = 30; @@ -43,21 +92,8 @@ public class ConfigConstants { /* one hour interval */ public static final int PROXY_HTTP_UPDATE_INTERVAL_MINUTES = 60; - public static final int PROXY_UPDATE_MAX_RETRY = 10; - public static final int MAX_LINE_CNT = 30; - // connection timeout in milliseconds - public static final long VAL_DEF_CONNECT_TIMEOUT_MS = 20000L; - public static final long VAL_MIN_CONNECT_TIMEOUT_MS = 1L; - public static final long VAL_DEF_CONNECT_CLOSE_DELAY_MS = 500L; - // request timeout in milliseconds - public static final long VAL_DEF_REQUEST_TIMEOUT_MS = 10000L; - public static final long VAL_MIN_REQUEST_TIMEOUT_MS = 1L; - - public static final int DEFAULT_SEND_BUFFER_SIZE = 16777216; - public static final int DEFAULT_RECEIVE_BUFFER_SIZE = 16777216; - public static final String RECEIVE_BUFFER_SIZE = "receiveBufferSize"; public static final String SEND_BUFFER_SIZE = "sendBufferSize"; @@ -65,14 +101,6 @@ public class ConfigConstants { public static final int FLAG_ALLOW_ENCRYPT = 1 << 6; public static final int FLAG_ALLOW_COMPRESS = 1 << 5; - public static final String MANAGER_DATAPROXY_API = "/inlong/manager/openapi/dataproxy/getIpList/"; - public static LoadBalance DEFAULT_LOAD_BALANCE = LoadBalance.ROBIN; - public static int DEFAULT_VIRTUAL_NODE = 1000; - public static int DEFAULT_RANDOM_MAX_RETRY = 1000; - - public static String HTTP = "http://"; - public static String HTTPS = "https://"; - public static int DEFAULT_SENDER_MAX_ATTEMPT = 1; /* Reserved attribute data size(bytes). */ diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/DefaultMessageSender.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/DefaultMessageSender.java index 153c43b8db0..61a68e0d9c3 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/DefaultMessageSender.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/DefaultMessageSender.java @@ -28,10 +28,9 @@ import org.apache.inlong.sdk.dataproxy.network.ProxysdkException; import org.apache.inlong.sdk.dataproxy.network.Sender; import org.apache.inlong.sdk.dataproxy.network.SequentialID; -import org.apache.inlong.sdk.dataproxy.network.Utils; import org.apache.inlong.sdk.dataproxy.threads.IndexCollectThread; -import org.apache.inlong.sdk.dataproxy.threads.ManagerFetcherThread; import org.apache.inlong.sdk.dataproxy.utils.ProxyUtils; +import org.apache.inlong.sdk.dataproxy.utils.Tuple2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,8 +52,7 @@ public class DefaultMessageSender implements MessageSender { private static final ConcurrentHashMap CACHE_SENDER = new ConcurrentHashMap<>(); private static final AtomicBoolean MANAGER_FETCHER_THREAD_STARTED = new AtomicBoolean(false); - private static ManagerFetcherThread managerFetcherThread; - private static final SequentialID idGenerator = new SequentialID(Utils.getLocalIp()); + private static final SequentialID idGenerator = new SequentialID(); private final Sender sender; private final IndexCollectThread indexCol; /* Store index */ @@ -65,7 +63,7 @@ public class DefaultMessageSender implements MessageSender { private boolean isGroupIdTransfer = false; private boolean isReport = false; private boolean isSupportLF = false; - private int maxPacketLength; + private int maxPacketLength = -1; private int cpsSize = ConfigConstants.COMPRESS_SIZE; private final int senderMaxAttempt; @@ -76,6 +74,7 @@ public DefaultMessageSender(ProxyClientConfig configure) throws Exception { public DefaultMessageSender(ProxyClientConfig configure, ThreadFactory selfDefineFactory) throws Exception { ProxyUtils.validClientConfig(configure); sender = new Sender(configure, selfDefineFactory); + sender.start(); groupId = configure.getInlongGroupId(); indexCol = new IndexCollectThread(storeIndex); senderMaxAttempt = configure.getSenderMaxAttempt(); @@ -110,18 +109,20 @@ public static DefaultMessageSender generateSenderByClusterId(ProxyClientConfig c } LOGGER.info("Initial tcp sender, configure is {}", configure); // initial sender object - ProxyConfigManager proxyConfigManager = new ProxyConfigManager(configure, - Utils.getLocalIp(), null); - proxyConfigManager.setInlongGroupId(configure.getInlongGroupId()); - ProxyConfigEntry entry = proxyConfigManager.getGroupIdConfigure(); - DefaultMessageSender sender = CACHE_SENDER.get(entry.getClusterId()); + ProxyConfigManager proxyConfigManager = new ProxyConfigManager(configure); + Tuple2 result = + proxyConfigManager.getGroupIdConfigure(true); + if (result.getF0() == null) { + throw new Exception(result.getF1()); + } + DefaultMessageSender sender = CACHE_SENDER.get(result.getF0().getClusterId()); if (sender != null) { return sender; } else { DefaultMessageSender tmpMessageSender = new DefaultMessageSender(configure, selfDefineFactory); - tmpMessageSender.setMaxPacketLength(entry.getMaxPacketLength()); - CACHE_SENDER.put(entry.getClusterId(), tmpMessageSender); + tmpMessageSender.setMaxPacketLength(result.getF0().getMaxPacketLength()); + CACHE_SENDER.put(result.getF0().getClusterId(), tmpMessageSender); return tmpMessageSender; } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/ProxyClientConfig.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/ProxyClientConfig.java index 3338d866c24..7af91dba34e 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/ProxyClientConfig.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/ProxyClientConfig.java @@ -18,8 +18,8 @@ package org.apache.inlong.sdk.dataproxy; import org.apache.inlong.sdk.dataproxy.metric.MetricConfig; +import org.apache.inlong.sdk.dataproxy.network.IpUtils; import org.apache.inlong.sdk.dataproxy.network.ProxysdkException; -import org.apache.inlong.sdk.dataproxy.network.Utils; import lombok.Data; import org.apache.commons.lang3.StringUtils; @@ -27,43 +27,40 @@ @Data public class ProxyClientConfig { - private int aliveConnections; + private String managerIP = ""; + private int managerPort = 8099; + private boolean visitManagerByHttp = true; + private boolean onlyUseLocalProxyConfig = false; + private int managerConnTimeoutMs = ConfigConstants.VAL_DEF_CONNECT_TIMEOUT_MS; + private int managerSocketTimeoutMs = ConfigConstants.VAL_DEF_SOCKET_TIMEOUT_MS; + private long managerConfigSyncInrMs = + ConfigConstants.VAL_DEF_CONFIG_SYNC_INTERVAL_MIN * ConfigConstants.VAL_UNIT_MIN_TO_MS; + private int configSyncMaxRetryIfFail = ConfigConstants.VAL_DEF_RETRY_IF_CONFIG_SYNC_FAIL; + private String configStoreBasePath = System.getProperty("user.dir"); + // max expired time for config cache. + private long configCacheExpiredMs = ConfigConstants.VAL_DEF_CACHE_CONFIG_EXPIRED_MS; + // nodes force choose interval ms + private long forceReChooseInrMs = ConfigConstants.VAL_DEF_FORCE_CHOOSE_INR_MS; + private boolean enableAuthentication = false; + private String authSecretId = ""; + private String authSecretKey = ""; + private String inlongGroupId; + private int aliveConnections = ConfigConstants.VAL_DEF_ALIVE_CONNECTIONS; + private int syncThreadPoolSize; private int asyncCallbackSize; - private int managerPort = 8099; - private String managerIP = ""; - private String managerAddress; - private String managerIpLocalPath = System.getProperty("user.dir") + "/.inlong/.managerIps"; - private String managerUrl = ""; - private int proxyUpdateIntervalMinutes; - private int proxyUpdateMaxRetry; - private String inlongGroupId; - private boolean requestByHttp = true; private boolean isNeedDataEncry = false; - private boolean needAuthentication = false; - private String userName = ""; - private String secretKey = ""; private String rsaPubKeyUrl = ""; - private String confStoreBasePath = System.getProperty("user.dir") + "/.inlong/"; private String tlsServerCertFilePathAndName; private String tlsServerKey; private String tlsVersion = "TLSv1.2"; private int maxTimeoutCnt = ConfigConstants.MAX_TIMEOUT_CNT; - private String authSecretId; - private String authSecretKey; private String protocolType; - private boolean enableSaveManagerVIps = false; // metric configure private MetricConfig metricConfig = new MetricConfig(); - private int managerConnectionTimeout = 10000; - // http socket timeout in milliseconds - private int managerSocketTimeout = 30 * 1000; - - private boolean readProxyIPFromLocal = false; - // connect timeout in milliseconds private long connectTimeoutMs = ConfigConstants.VAL_DEF_CONNECT_TIMEOUT_MS; // request timeout in milliseconds @@ -71,6 +68,14 @@ public class ProxyClientConfig { // connect close wait period in milliseconds private long conCloseWaitPeriodMs = ConfigConstants.VAL_DEF_REQUEST_TIMEOUT_MS + ConfigConstants.VAL_DEF_CONNECT_CLOSE_DELAY_MS; + // client reconnect wait period in ms + private long reConnectWaitMs = ConfigConstants.VAL_DEF_RECONNECT_WAIT_MS; + // socket receive buffer + private int recvBufferSize = ConfigConstants.DEFAULT_RECEIVE_BUFFER_SIZE; + // socket send buffer + private int sendBufferSize = ConfigConstants.DEFAULT_SEND_BUFFER_SIZE; + // max message count per connection + private long maxMsgInFlightPerConn = ConfigConstants.MAX_INFLIGHT_MSG_COUNT_PER_CONNECTION; // configuration for http client // whether discard old metric when cache is full. @@ -81,164 +86,171 @@ public class ProxyClientConfig { // interval for async worker in microseconds. private int asyncWorkerInterval = 500; private boolean cleanHttpCacheWhenClosing = false; - // max cache time for proxy config. - private long maxProxyCacheTimeInMs = 30 * 60 * 1000; private int ioThreadNum = Runtime.getRuntime().availableProcessors(); private boolean enableBusyWait = false; - private int virtualNode; - - private LoadBalance loadBalance; - - private int maxRetry; private int senderMaxAttempt = ConfigConstants.DEFAULT_SENDER_MAX_ATTEMPT; /* pay attention to the last url parameter ip */ - public ProxyClientConfig(String localHost, boolean requestByHttp, String managerIp, - int managerPort, String inlongGroupId, String authSecretId, String authSecretKey, - LoadBalance loadBalance, int virtualNode, int maxRetry) throws ProxysdkException { - if (Utils.isBlank(localHost)) { + public ProxyClientConfig(String localHost, boolean visitManagerByHttp, String managerIp, + int managerPort, String inlongGroupId, String authSecretId, String authSecretKey) throws ProxysdkException { + if (StringUtils.isBlank(localHost)) { throw new ProxysdkException("localHost is blank!"); } - if (Utils.isBlank(managerIp)) { - throw new IllegalArgumentException("managerIp is Blank!"); + if (StringUtils.isBlank(managerIp)) { + throw new ProxysdkException("managerIp is Blank!"); } - if (Utils.isBlank(inlongGroupId)) { + if (managerPort <= 0) { + throw new ProxysdkException("managerPort <= 0!"); + } + if (StringUtils.isBlank(inlongGroupId)) { throw new ProxysdkException("groupId is blank!"); } - this.inlongGroupId = inlongGroupId; - this.requestByHttp = requestByHttp; + this.inlongGroupId = inlongGroupId.trim(); + this.visitManagerByHttp = visitManagerByHttp; this.managerPort = managerPort; this.managerIP = managerIp; - this.managerAddress = getManagerAddress(managerIp, managerPort, requestByHttp); - this.managerUrl = - getManagerUrl(managerAddress, inlongGroupId); - Utils.validLocalIp(localHost); - this.aliveConnections = ConfigConstants.ALIVE_CONNECTIONS; + IpUtils.validLocalIp(localHost); this.syncThreadPoolSize = ConfigConstants.SYNC_THREAD_POOL_SIZE; this.asyncCallbackSize = ConfigConstants.ASYNC_CALLBACK_SIZE; - this.proxyUpdateIntervalMinutes = ConfigConstants.PROXY_UPDATE_INTERVAL_MINUTES; this.proxyHttpUpdateIntervalMinutes = ConfigConstants.PROXY_HTTP_UPDATE_INTERVAL_MINUTES; - this.proxyUpdateMaxRetry = ConfigConstants.PROXY_UPDATE_MAX_RETRY; this.authSecretId = authSecretId; this.authSecretKey = authSecretKey; - this.loadBalance = loadBalance; - this.virtualNode = virtualNode; - this.maxRetry = maxRetry; } /* pay attention to the last url parameter ip */ - public ProxyClientConfig(String managerAddress, String inlongGroupId, String authSecretId, String authSecretKey, - LoadBalance loadBalance, int virtualNode, int maxRetry) throws ProxysdkException { - if (Utils.isBlank(managerAddress) || (!managerAddress.startsWith(ConfigConstants.HTTP) - && !managerAddress.startsWith(ConfigConstants.HTTPS))) { - throw new ProxysdkException("managerAddress is blank or missing http/https protocol "); - } - if (Utils.isBlank(inlongGroupId)) { + public ProxyClientConfig(String managerAddress, + String inlongGroupId, String authSecretId, String authSecretKey) throws ProxysdkException { + checkAndParseAddress(managerAddress); + if (StringUtils.isBlank(inlongGroupId)) { throw new ProxysdkException("groupId is blank!"); } - if (managerAddress.startsWith(ConfigConstants.HTTPS)) { - this.requestByHttp = false; - } - this.managerAddress = managerAddress; - this.managerUrl = getManagerUrl(managerAddress, inlongGroupId); - this.inlongGroupId = inlongGroupId; - this.aliveConnections = ConfigConstants.ALIVE_CONNECTIONS; + this.inlongGroupId = inlongGroupId.trim(); this.syncThreadPoolSize = ConfigConstants.SYNC_THREAD_POOL_SIZE; this.asyncCallbackSize = ConfigConstants.ASYNC_CALLBACK_SIZE; - this.proxyUpdateIntervalMinutes = ConfigConstants.PROXY_UPDATE_INTERVAL_MINUTES; this.proxyHttpUpdateIntervalMinutes = ConfigConstants.PROXY_HTTP_UPDATE_INTERVAL_MINUTES; - this.proxyUpdateMaxRetry = ConfigConstants.PROXY_UPDATE_MAX_RETRY; this.authSecretId = authSecretId; this.authSecretKey = authSecretKey; - this.loadBalance = loadBalance; - this.virtualNode = virtualNode; - this.maxRetry = maxRetry; } - private String getManagerUrl(String managerAddress, String inlongGroupId) { - return managerAddress + ConfigConstants.MANAGER_DATAPROXY_API + inlongGroupId; + public String getManagerIP() { + return managerIP; } - private String getManagerAddress(String managerIp, int managerPort, boolean requestByHttp) { - String protocolType = ConfigConstants.HTTPS; - if (requestByHttp) { - protocolType = ConfigConstants.HTTP; - } - return protocolType + managerIp + ":" + managerPort; + public int getManagerPort() { + return managerPort; } - public ProxyClientConfig(String localHost, boolean requestByHttp, String managerIp, int managerPort, - String inlongGroupId, String authSecretId, String authSecretKey) throws ProxysdkException { - this(localHost, requestByHttp, managerIp, managerPort, inlongGroupId, authSecretId, authSecretKey, - ConfigConstants.DEFAULT_LOAD_BALANCE, ConfigConstants.DEFAULT_VIRTUAL_NODE, - ConfigConstants.DEFAULT_RANDOM_MAX_RETRY); + public boolean isVisitManagerByHttp() { + return visitManagerByHttp; } - public ProxyClientConfig(String managerAddress, String inlongGroupId, String authSecretId, String authSecretKey) - throws ProxysdkException { - this(managerAddress, inlongGroupId, authSecretId, authSecretKey, - ConfigConstants.DEFAULT_LOAD_BALANCE, ConfigConstants.DEFAULT_VIRTUAL_NODE, - ConfigConstants.DEFAULT_RANDOM_MAX_RETRY); + public boolean isOnlyUseLocalProxyConfig() { + return onlyUseLocalProxyConfig; } - public String getTlsServerCertFilePathAndName() { - return tlsServerCertFilePathAndName; + public void setOnlyUseLocalProxyConfig(boolean onlyUseLocalProxyConfig) { + this.onlyUseLocalProxyConfig = onlyUseLocalProxyConfig; } - public String getTlsServerKey() { - return tlsServerKey; + public boolean isEnableAuthentication() { + return this.enableAuthentication; } - public boolean isRequestByHttp() { - return requestByHttp; + public String getAuthSecretId() { + return authSecretId; } - public String getInlongGroupId() { - return inlongGroupId; + public String getAuthSecretKey() { + return authSecretKey; } - public void setInlongGroupId(String inlongGroupId) { - this.inlongGroupId = inlongGroupId; + public void setAuthenticationInfo(boolean needAuthentication, String secretId, String secretKey) { + this.enableAuthentication = needAuthentication; + if (!this.enableAuthentication) { + return; + } + if (StringUtils.isBlank(secretId)) { + throw new IllegalArgumentException("secretId is Blank!"); + } + if (StringUtils.isBlank(secretKey)) { + throw new IllegalArgumentException("secretKey is Blank!"); + } + this.authSecretId = secretId.trim(); + this.authSecretKey = secretKey.trim(); } - public int getManagerPort() { - return managerPort; + public long getManagerConfigSyncInrMs() { + return managerConfigSyncInrMs; } - public String getManagerIP() { - return managerIP; + public void setManagerConfigSyncInrMin(int managerConfigSyncInrMin) { + int tmpValue = + Math.min(ConfigConstants.VAL_MAX_CONFIG_SYNC_INTERVAL_MIN, + Math.max(ConfigConstants.VAL_MIN_CONFIG_SYNC_INTERVAL_MIN, managerConfigSyncInrMin)); + this.managerConfigSyncInrMs = tmpValue * ConfigConstants.VAL_UNIT_MIN_TO_MS; } - public String getManagerIpLocalPath() { - return managerIpLocalPath; + public int getManagerConnTimeoutMs() { + return managerConnTimeoutMs; } - public void setManagerIpLocalPath(String managerIpLocalPath) throws ProxysdkException { - if (StringUtils.isEmpty(managerIpLocalPath)) { - throw new ProxysdkException("managerIpLocalPath is empty."); - } - if (managerIpLocalPath.charAt(managerIpLocalPath.length() - 1) == '/') { - managerIpLocalPath = managerIpLocalPath.substring(0, managerIpLocalPath.length() - 1); + public void setManagerConnTimeoutMs(int managerConnTimeoutMs) { + this.managerConnTimeoutMs = + Math.min(ConfigConstants.VAL_MAX_CONNECT_TIMEOUT_MS, + Math.max(ConfigConstants.VAL_MIN_CONNECT_TIMEOUT_MS, managerConnTimeoutMs)); + } + + public int getManagerSocketTimeoutMs() { + return managerSocketTimeoutMs; + } + + public void setManagerSocketTimeoutMs(int managerSocketTimeoutMs) { + this.managerSocketTimeoutMs = + Math.min(ConfigConstants.VAL_MAX_SOCKET_TIMEOUT_MS, + Math.max(ConfigConstants.VAL_MIN_SOCKET_TIMEOUT_MS, managerSocketTimeoutMs)); + } + + public int getConfigSyncMaxRetryIfFail() { + return configSyncMaxRetryIfFail; + } + + public void setConfigSyncMaxRetryIfFail(int configSyncMaxRetryIfFail) { + this.configSyncMaxRetryIfFail = + Math.min(configSyncMaxRetryIfFail, ConfigConstants.VAL_MAX_RETRY_IF_CONFIG_SYNC_FAIL); + } + + public String getConfigStoreBasePath() { + return configStoreBasePath; + } + + public void setConfigStoreBasePath(String configStoreBasePath) { + if (StringUtils.isBlank(configStoreBasePath)) { + return; } - this.managerIpLocalPath = managerIpLocalPath + "/.managerIps"; + this.configStoreBasePath = configStoreBasePath.trim(); + } + + public long getConfigCacheExpiredMs() { + return configCacheExpiredMs; } - public boolean isEnableSaveManagerVIps() { - return enableSaveManagerVIps; + public void setConfigCacheExpiredMs(long configCacheExpiredMs) { + this.configCacheExpiredMs = configCacheExpiredMs; } - public void setEnableSaveManagerVIps(boolean enable) { - this.enableSaveManagerVIps = enable; + public long getForceReChooseInrMs() { + return forceReChooseInrMs; } - public String getConfStoreBasePath() { - return confStoreBasePath; + public void setForceReChooseInrMs(long forceReChooseInrMs) { + this.forceReChooseInrMs = + Math.max(ConfigConstants.VAL_MIN_FORCE_CHOOSE_INR_MS, forceReChooseInrMs); } - public void setConfStoreBasePath(String confStoreBasePath) { - this.confStoreBasePath = confStoreBasePath; + public String getInlongGroupId() { + return inlongGroupId; } public int getAliveConnections() { @@ -246,7 +258,16 @@ public int getAliveConnections() { } public void setAliveConnections(int aliveConnections) { - this.aliveConnections = aliveConnections; + this.aliveConnections = + Math.max(ConfigConstants.VAL_MIN_ALIVE_CONNECTIONS, aliveConnections); + } + + public String getTlsServerCertFilePathAndName() { + return tlsServerCertFilePathAndName; + } + + public String getTlsServerKey() { + return tlsServerKey; } public int getSyncThreadPoolSize() { @@ -268,10 +289,6 @@ public void setTotalAsyncCallbackSize(int asyncCallbackSize) { this.asyncCallbackSize = asyncCallbackSize; } - public String getManagerUrl() { - return managerUrl; - } - public int getMaxTimeoutCnt() { return maxTimeoutCnt; } @@ -283,22 +300,6 @@ public void setMaxTimeoutCnt(int maxTimeoutCnt) { this.maxTimeoutCnt = maxTimeoutCnt; } - public int getProxyUpdateIntervalMinutes() { - return proxyUpdateIntervalMinutes; - } - - public void setProxyUpdateIntervalMinutes(int proxyUpdateIntervalMinutes) { - this.proxyUpdateIntervalMinutes = proxyUpdateIntervalMinutes; - } - - public int getProxyUpdateMaxRetry() { - return proxyUpdateMaxRetry; - } - - public void setProxyUpdateMaxRetry(int proxyUpdateMaxRetry) { - this.proxyUpdateMaxRetry = proxyUpdateMaxRetry; - } - public long getConnectTimeoutMs() { return connectTimeoutMs; } @@ -331,39 +332,57 @@ public void setConCloseWaitPeriodMs(long conCloseWaitPeriodMs) { } } - public String getRsaPubKeyUrl() { - return rsaPubKeyUrl; + public long getReConnectWaitMs() { + return reConnectWaitMs; } - public boolean isNeedDataEncry() { - return isNeedDataEncry; + public void setReConnectWaitMs(long reConnectWaitMs) { + if (reConnectWaitMs > ConfigConstants.VAL_MAX_RECONNECT_WAIT_MS) { + this.reConnectWaitMs = ConfigConstants.VAL_MAX_RECONNECT_WAIT_MS; + } } - public boolean isNeedAuthentication() { - return this.needAuthentication; + public int getRecvBufferSize() { + return recvBufferSize; } - public void setAuthenticationInfo(boolean needAuthentication, boolean needDataEncry, - final String userName, final String secretKey) { - this.needAuthentication = needAuthentication; - this.isNeedDataEncry = needDataEncry; - if (this.needAuthentication || this.isNeedDataEncry) { - if (Utils.isBlank(userName)) { - throw new IllegalArgumentException("userName is Blank!"); - } - if (Utils.isBlank(secretKey)) { - throw new IllegalArgumentException("secretKey is Blank!"); - } + public void setRecvBufferSize(int recvBufferSize) { + if (recvBufferSize > 0 && recvBufferSize < Integer.MAX_VALUE) { + this.recvBufferSize = recvBufferSize; } - this.userName = userName.trim(); - this.secretKey = secretKey.trim(); + } + + public int getSendBufferSize() { + return sendBufferSize; + } + + public void setSendBufferSize(int sendBufferSize) { + if (sendBufferSize > 0 && sendBufferSize < Integer.MAX_VALUE) { + this.sendBufferSize = sendBufferSize; + } + } + + public long getMaxMsgInFlightPerConn() { + return maxMsgInFlightPerConn; + } + + public void setMaxMsgInFlightPerConn(long maxMsgInFlightPerConn) { + this.maxMsgInFlightPerConn = maxMsgInFlightPerConn; + } + + public String getRsaPubKeyUrl() { + return rsaPubKeyUrl; + } + + public boolean isNeedDataEncry() { + return isNeedDataEncry; } public void setHttpsInfo(String tlsServerCertFilePathAndName, String tlsServerKey) { - if (Utils.isBlank(tlsServerCertFilePathAndName)) { + if (StringUtils.isBlank(tlsServerCertFilePathAndName)) { throw new IllegalArgumentException("tlsServerCertFilePathAndName is Blank!"); } - if (Utils.isBlank(tlsServerKey)) { + if (StringUtils.isBlank(tlsServerKey)) { throw new IllegalArgumentException("tlsServerKey is Blank!"); } this.tlsServerKey = tlsServerKey; @@ -378,22 +397,6 @@ public void setTlsVersion(String tlsVersion) { this.tlsVersion = tlsVersion; } - public String getUserName() { - return userName; - } - - public String getSecretKey() { - return secretKey; - } - - public boolean isReadProxyIPFromLocal() { - return readProxyIPFromLocal; - } - - public void setReadProxyIPFromLocal(boolean readProxyIPFromLocal) { - this.readProxyIPFromLocal = readProxyIPFromLocal; - } - public int getProxyHttpUpdateIntervalMinutes() { return proxyHttpUpdateIntervalMinutes; } @@ -426,14 +429,6 @@ public void setAsyncWorkerInterval(int asyncWorkerInterval) { this.asyncWorkerInterval = asyncWorkerInterval; } - public int getManagerSocketTimeout() { - return managerSocketTimeout; - } - - public void setManagerSocketTimeout(int managerSocketTimeout) { - this.managerSocketTimeout = managerSocketTimeout; - } - public boolean isCleanHttpCacheWhenClosing() { return cleanHttpCacheWhenClosing; } @@ -442,22 +437,6 @@ public void setCleanHttpCacheWhenClosing(boolean cleanHttpCacheWhenClosing) { this.cleanHttpCacheWhenClosing = cleanHttpCacheWhenClosing; } - public long getMaxProxyCacheTimeInMs() { - return maxProxyCacheTimeInMs; - } - - public void setMaxProxyCacheTimeInMs(long maxProxyCacheTimeInMs) { - this.maxProxyCacheTimeInMs = maxProxyCacheTimeInMs; - } - - public int getManagerConnectionTimeout() { - return managerConnectionTimeout; - } - - public void setManagerConnectionTimeout(int managerConnectionTimeout) { - this.managerConnectionTimeout = managerConnectionTimeout; - } - public MetricConfig getMetricConfig() { return metricConfig; } @@ -489,29 +468,6 @@ public void setEnableBusyWait(boolean enableBusyWait) { this.enableBusyWait = enableBusyWait; } - public int getVirtualNode() { - return virtualNode; - } - - public void setVirtualNode(int virtualNode) { - this.virtualNode = virtualNode; - } - - public LoadBalance getLoadBalance() { - return loadBalance; - } - - public void setLoadBalance(LoadBalance loadBalance) { - this.loadBalance = loadBalance; - } - - public int getMaxRetry() { - return maxRetry; - } - - public void setMaxRetry(int maxRetry) { - this.maxRetry = maxRetry; - } public int getSenderMaxAttempt() { return senderMaxAttempt; } @@ -519,4 +475,40 @@ public int getSenderMaxAttempt() { public void setSenderAttempt(int senderMaxAttempt) { this.senderMaxAttempt = senderMaxAttempt; } + + private void checkAndParseAddress(String managerAddress) throws ProxysdkException { + if (StringUtils.isBlank(managerAddress) + || (!managerAddress.startsWith(ConfigConstants.HTTP) + && !managerAddress.startsWith(ConfigConstants.HTTPS))) { + throw new ProxysdkException("managerAddress is blank or missing http/https protocol"); + } + String hostPortInfo; + if (managerAddress.startsWith(ConfigConstants.HTTPS)) { + this.visitManagerByHttp = false; + hostPortInfo = managerAddress.substring(ConfigConstants.HTTPS.length() + 1); + } else { + hostPortInfo = managerAddress.substring(ConfigConstants.HTTP.length() + 1); + } + if (StringUtils.isBlank(hostPortInfo)) { + throw new ProxysdkException("managerAddress must include host:port info!"); + } + String[] fields = hostPortInfo.split(":"); + if (fields.length == 1) { + throw new ProxysdkException("managerAddress must include port info!"); + } else if (fields.length > 2) { + throw new ProxysdkException("managerAddress must only include host:port info!"); + } + if (StringUtils.isBlank(fields[0])) { + throw new ProxysdkException("managerAddress's host is blank!"); + } + this.managerIP = fields[0].trim(); + if (StringUtils.isBlank(fields[1])) { + throw new ProxysdkException("managerAddress's port is blank!"); + } + try { + this.managerPort = Integer.parseInt(fields[1]); + } catch (Throwable ex) { + throw new ProxysdkException("managerAddress's port must be number!"); + } + } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/EncodeObject.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/EncodeObject.java index a89fef4fe95..6e00e98158e 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/EncodeObject.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/EncodeObject.java @@ -53,8 +53,8 @@ public class EncodeObject { private boolean isAuth = false; private boolean isEncrypt = false; private boolean isCompress = true; - private int groupIdNum; - private int streamIdNum; + private int groupIdNum = 0; + private int streamIdNum = 0; private String groupId; private String streamId; private short load; @@ -210,8 +210,20 @@ public boolean isGroupIdTransfer() { return isGroupIdTransfer; } - public void setGroupIdTransfer(boolean isGroupIdTransfer) { - this.isGroupIdTransfer = isGroupIdTransfer; + public int getGroupIdNum() { + return groupIdNum; + } + + public int getStreamIdNum() { + return streamIdNum; + } + + public void setGroupIdAndStreamIdNum(int groupIdNum, int streamIdNum) { + this.groupIdNum = groupIdNum; + this.streamIdNum = streamIdNum; + if (groupIdNum != 0 && streamIdNum != 0) { + this.isGroupIdTransfer = true; + } } public short getLoad() { @@ -280,22 +292,6 @@ public void setEncryptEntry(boolean isEncrypt, String userName, EncryptConfigEnt this.encryptEntry = encryptEntry; } - public int getGroupIdNum() { - return groupIdNum; - } - - public void setGroupIdNum(int groupIdNum) { - this.groupIdNum = groupIdNum; - } - - public int getStreamIdNum() { - return streamIdNum; - } - - public void setStreamIdNum(int streamIdNum) { - this.streamIdNum = streamIdNum; - } - public long getDt() { return dt; } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/ProtocolDecoder.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/ProtocolDecoder.java index 2038a8b8d68..b7c31cba79e 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/ProtocolDecoder.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/ProtocolDecoder.java @@ -17,6 +17,8 @@ package org.apache.inlong.sdk.dataproxy.codec; +import org.apache.inlong.sdk.dataproxy.utils.LogCounter; + import io.netty.buffer.ByteBuf; import io.netty.channel.ChannelHandlerContext; import io.netty.handler.codec.MessageToMessageDecoder; @@ -29,18 +31,19 @@ public class ProtocolDecoder extends MessageToMessageDecoder { - private static final Logger LOGGER = LoggerFactory.getLogger(ProtocolDecoder.class); - + private static final Logger logger = LoggerFactory.getLogger(ProtocolDecoder.class); + private static final LogCounter decExptCounter = new LogCounter(10, 200000, 60 * 1000L); @Override protected void decode(ChannelHandlerContext ctx, ByteBuf buffer, List out) throws Exception { buffer.markReaderIndex(); // totallen int totalLen = buffer.readInt(); - LOGGER.debug("decode totalLen : {}", totalLen); if (totalLen != buffer.readableBytes()) { - LOGGER.error("totalLen is not equal readableBytes.total:" + totalLen - + ";readableBytes:" + buffer.readableBytes()); + if (decExptCounter.shouldPrint()) { + logger.error("Length not equal, totalLen={},readableBytes={},from={}", + totalLen, buffer.readableBytes(), ctx.channel()); + } buffer.resetReaderIndex(); throw new Exception("totalLen is not equal readableBytes.total"); } @@ -48,14 +51,17 @@ protected void decode(ChannelHandlerContext ctx, int msgType = buffer.readByte() & 0x1f; if (msgType == 4) { - LOGGER.info("debug decode"); - } - if (msgType == 3 | msgType == 5) { + if (logger.isDebugEnabled()) { + logger.debug("debug decode"); + } + } else if (msgType == 3 | msgType == 5) { // bodylen int bodyLength = buffer.readInt(); if (bodyLength >= totalLen) { - LOGGER.error("bodyLen is greater than totalLen.totalLen:" + totalLen - + ";bodyLen:" + bodyLength); + if (decExptCounter.shouldPrint()) { + logger.error("bodyLen greater than totalLen, totalLen={},bodyLen={},from={}", + totalLen, bodyLength, ctx.channel()); + } buffer.resetReaderIndex(); throw new Exception("bodyLen is greater than totalLen.totalLen"); } @@ -64,20 +70,19 @@ protected void decode(ChannelHandlerContext ctx, bodyBytes = new byte[bodyLength]; buffer.readBytes(bodyBytes); } - // attrlen + String attrInfo = ""; int attrLength = buffer.readInt(); - byte[] attrBytes = null; if (attrLength > 0) { - attrBytes = new byte[attrLength]; + byte[] attrBytes = new byte[attrLength]; buffer.readBytes(attrBytes); + attrInfo = new String(attrBytes, StandardCharsets.UTF_8); } EncodeObject object; if (bodyBytes == null) { - object = new EncodeObject(new String(attrBytes, StandardCharsets.UTF_8)); + object = new EncodeObject(attrInfo); } else { - object = new EncodeObject(Collections.singletonList(bodyBytes), - new String(attrBytes, StandardCharsets.UTF_8)); + object = new EncodeObject(Collections.singletonList(bodyBytes), attrInfo); } object.setMsgtype(5); out.add(object); @@ -85,12 +90,13 @@ protected void decode(ChannelHandlerContext ctx, int seqId = buffer.readInt(); int attrLen = buffer.readShort(); - byte[] attrBytes = null; + String attrInfo = ""; if (attrLen > 0) { - attrBytes = new byte[attrLen]; + byte[] attrBytes = new byte[attrLen]; buffer.readBytes(attrBytes); + attrInfo = new String(attrBytes, StandardCharsets.UTF_8); } - EncodeObject object = new EncodeObject(new String(attrBytes, StandardCharsets.UTF_8)); + EncodeObject object = new EncodeObject(attrInfo); object.setMessageId(String.valueOf(seqId)); buffer.readShort(); @@ -103,15 +109,14 @@ protected void decode(ChannelHandlerContext ctx, buffer.skipBytes(4 + 1 + 4); // skip datatime, body_ver and body_len final short load = buffer.readShort(); // read from body int attrLen = buffer.readShort(); - byte[] attrBytes = null; + String attrInfo = ""; if (attrLen > 0) { - attrBytes = new byte[attrLen]; + byte[] attrBytes = new byte[attrLen]; buffer.readBytes(attrBytes); + attrInfo = new String(attrBytes, StandardCharsets.UTF_8); } buffer.skipBytes(2); // skip magic - - String attrs = (attrBytes == null ? "" : new String(attrBytes, StandardCharsets.UTF_8)); - EncodeObject object = new EncodeObject(attrs); + EncodeObject object = new EncodeObject(attrInfo); object.setMsgtype(8); object.setLoad(load); out.add(object); diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/ProtocolEncoder.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/ProtocolEncoder.java index ecc1e1de91e..1a20766e55e 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/ProtocolEncoder.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/codec/ProtocolEncoder.java @@ -20,7 +20,7 @@ import org.apache.inlong.common.msg.AttributeConstants; import org.apache.inlong.sdk.dataproxy.config.EncryptConfigEntry; import org.apache.inlong.sdk.dataproxy.config.EncryptInfo; -import org.apache.inlong.sdk.dataproxy.network.Utils; +import org.apache.inlong.sdk.dataproxy.network.IpUtils; import org.apache.inlong.sdk.dataproxy.utils.EncryptUtil; import org.apache.inlong.sdk.dataproxy.utils.LogCounter; @@ -28,6 +28,7 @@ import io.netty.buffer.ByteBufAllocator; import io.netty.channel.ChannelHandlerContext; import io.netty.handler.codec.MessageToMessageEncoder; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.xerial.snappy.Snappy; @@ -78,18 +79,18 @@ private ByteBuf writeToBuf8(EncodeObject object) { try { String endAttr = object.getCommonattr(); if (object.isAuth()) { - if (Utils.isNotBlank(endAttr)) { + if (StringUtils.isNotBlank(endAttr)) { endAttr = endAttr + "&"; } long timestamp = System.currentTimeMillis(); int nonce = new SecureRandom(String.valueOf(timestamp).getBytes()).nextInt(Integer.MAX_VALUE); - endAttr = endAttr + "_userName=" + object.getUserName() + "&_clientIP=" + Utils.getLocalIp() - + "&_signature=" + Utils.generateSignature(object.getUserName(), + endAttr = endAttr + "_userName=" + object.getUserName() + "&_clientIP=" + IpUtils.getLocalIp() + + "&_signature=" + IpUtils.generateSignature(object.getUserName(), timestamp, nonce, object.getSecretKey()) + "&_timeStamp=" + timestamp + "&_nonce=" + nonce; } - if (Utils.isNotBlank(object.getMsgUUID())) { - if (Utils.isNotBlank(endAttr)) { + if (StringUtils.isNotBlank(object.getMsgUUID())) { + if (StringUtils.isNotBlank(endAttr)) { endAttr = endAttr + "&"; } endAttr = endAttr + "msgUUID=" + object.getMsgUUID(); @@ -98,16 +99,17 @@ private ByteBuf writeToBuf8(EncodeObject object) { if (object.isAuth()) { msgType |= FLAG_ALLOW_AUTH; } - int totalLength = 1 + 4 + 1 + 4 + 2 + endAttr.getBytes("utf8").length + 2; + byte[] attrData = endAttr.getBytes(StandardCharsets.UTF_8); + int totalLength = 1 + 4 + 1 + 4 + 2 + attrData.length + 2; buf = ByteBufAllocator.DEFAULT.buffer(4 + totalLength); buf.writeInt(totalLength); buf.writeByte(msgType); buf.writeInt((int) object.getDt()); buf.writeByte(1); buf.writeInt(0); - buf.writeShort(endAttr.getBytes("utf8").length); - if (endAttr.getBytes("utf8").length > 0) { - buf.writeBytes(endAttr.getBytes("utf8")); + buf.writeShort(attrData.length); + if (attrData.length > 0) { + buf.writeBytes(attrData); } buf.writeShort(0xee01); } catch (Throwable ex) { @@ -129,7 +131,7 @@ private ByteBuf constructBody(byte[] body, EncodeObject object, if (object.isEncrypt()) { EncryptConfigEntry encryptEntry = object.getEncryptEntry(); if (encryptEntry != null) { - if (Utils.isNotBlank(endAttr)) { + if (StringUtils.isNotBlank(endAttr)) { endAttr = endAttr + "&"; } EncryptInfo encryptInfo = encryptEntry.getRsaEncryptInfo(); @@ -140,13 +142,13 @@ private ByteBuf constructBody(byte[] body, EncodeObject object, } } if (!object.isGroupIdTransfer()) { - if (Utils.isNotBlank(endAttr)) { + if (StringUtils.isNotBlank(endAttr)) { endAttr = endAttr + "&"; } endAttr = (endAttr + "groupId=" + object.getGroupId() + "&streamId=" + object.getStreamId()); } - if (Utils.isNotBlank(object.getMsgUUID())) { - if (Utils.isNotBlank(endAttr)) { + if (StringUtils.isNotBlank(object.getMsgUUID())) { + if (StringUtils.isNotBlank(endAttr)) { endAttr = endAttr + "&"; } endAttr = endAttr + "msgUUID=" + object.getMsgUUID(); @@ -159,7 +161,8 @@ private ByteBuf constructBody(byte[] body, EncodeObject object, if (object.isCompress()) { msgType |= FLAG_ALLOW_COMPRESS; } - totalLength = totalLength + body.length + endAttr.getBytes("utf8").length; + byte[] attrData = endAttr.getBytes(StandardCharsets.UTF_8); + totalLength = totalLength + body.length + attrData.length; buf = ByteBufAllocator.DEFAULT.buffer(4 + totalLength); buf.writeInt(totalLength); buf.writeByte(msgType); @@ -180,8 +183,8 @@ private ByteBuf constructBody(byte[] body, EncodeObject object, buf.writeInt(body.length); buf.writeBytes(body); - buf.writeShort(endAttr.getBytes("utf8").length); - buf.writeBytes(endAttr.getBytes("utf8")); + buf.writeShort(attrData.length); + buf.writeBytes(attrData); buf.writeShort(0xee01); } return buf; @@ -206,7 +209,7 @@ private ByteBuf writeToBuf7(EncodeObject object) { ByteArrayOutputStream data = new ByteArrayOutputStream(); for (byte[] entry : object.getBodylist()) { if (totalCnt++ > 0) { - data.write("\n".getBytes("utf8")); + data.write(AttributeConstants.LINE_FEED_SEP.getBytes(StandardCharsets.UTF_8)); } data.write(entry); } @@ -258,7 +261,7 @@ private ByteBuf writeToBuf5(EncodeObject object) { if (object.isEncrypt()) { EncryptConfigEntry encryptEntry = object.getEncryptEntry(); if (encryptEntry != null) { - if (Utils.isNotBlank(msgAttrs)) { + if (StringUtils.isNotBlank(msgAttrs)) { msgAttrs = msgAttrs + "&"; } EncryptInfo encryptInfo = encryptEntry.getRsaEncryptInfo(); @@ -268,8 +271,8 @@ private ByteBuf writeToBuf5(EncodeObject object) { body = EncryptUtil.aesEncrypt(body, encryptInfo.getAesKey()); } } - if (Utils.isNotBlank(object.getMsgUUID())) { - if (Utils.isNotBlank(msgAttrs)) { + if (StringUtils.isNotBlank(object.getMsgUUID())) { + if (StringUtils.isNotBlank(msgAttrs)) { msgAttrs = msgAttrs + "&"; } msgAttrs = msgAttrs + "msgUUID=" + object.getMsgUUID(); @@ -279,14 +282,15 @@ private ByteBuf writeToBuf5(EncodeObject object) { if (object.isEncrypt()) { msgType |= FLAG_ALLOW_ENCRYPT; } - totalLength = totalLength + body.length + msgAttrs.getBytes("utf8").length; + byte[] attrData = msgAttrs.getBytes(StandardCharsets.UTF_8); + totalLength = totalLength + body.length + attrData.length; buf = ByteBufAllocator.DEFAULT.buffer(4 + totalLength); buf.writeInt(totalLength); buf.writeByte(msgType); buf.writeInt(body.length); buf.writeBytes(body); - buf.writeInt(msgAttrs.getBytes("utf8").length); - buf.writeBytes(msgAttrs.getBytes("utf8")); + buf.writeInt(attrData.length); + buf.writeBytes(attrData); } } catch (Throwable ex) { if (exptCounter.shouldPrint()) { @@ -322,7 +326,7 @@ private ByteBuf writeToBuf3(EncodeObject object) { if (object.isEncrypt()) { EncryptConfigEntry encryptEntry = object.getEncryptEntry(); if (encryptEntry != null) { - if (Utils.isNotBlank(msgAttrs)) { + if (StringUtils.isNotBlank(msgAttrs)) { msgAttrs = msgAttrs + "&"; } EncryptInfo encryptInfo = encryptEntry.getRsaEncryptInfo(); @@ -332,8 +336,8 @@ private ByteBuf writeToBuf3(EncodeObject object) { body = EncryptUtil.aesEncrypt(body, encryptInfo.getAesKey()); } } - if (Utils.isNotBlank(object.getMsgUUID())) { - if (Utils.isNotBlank(msgAttrs)) { + if (StringUtils.isNotBlank(object.getMsgUUID())) { + if (StringUtils.isNotBlank(msgAttrs)) { msgAttrs = msgAttrs + "&"; } msgAttrs = msgAttrs + "msgUUID=" + object.getMsgUUID(); @@ -343,14 +347,15 @@ private ByteBuf writeToBuf3(EncodeObject object) { if (object.isEncrypt()) { msgType |= FLAG_ALLOW_ENCRYPT; } - totalLength = totalLength + body.length + msgAttrs.getBytes("utf8").length; + byte[] attrData = msgAttrs.getBytes(StandardCharsets.UTF_8); + totalLength = totalLength + body.length + attrData.length; buf = ByteBufAllocator.DEFAULT.buffer(4 + totalLength); buf.writeInt(totalLength); buf.writeByte(msgType); buf.writeInt(body.length); buf.writeBytes(body); - buf.writeInt(msgAttrs.getBytes("utf8").length); - buf.writeBytes(msgAttrs.getBytes("utf8")); + buf.writeInt(attrData.length); + buf.writeBytes(attrData); } } catch (Throwable ex) { if (exptCounter.shouldPrint()) { diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/common/SendResult.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/common/SendResult.java index adab601e0ac..97716b47231 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/common/SendResult.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/common/SendResult.java @@ -18,8 +18,8 @@ package org.apache.inlong.sdk.dataproxy.common; public enum SendResult { - INVALID_ATTRIBUTES, // including DataProxyErrCode(100,101,102,112) OK, + INVALID_ATTRIBUTES, // including DataProxyErrCode(100,101,102,112) TIMEOUT, CONNECTION_BREAK, THREAD_INTERRUPT, @@ -31,6 +31,13 @@ public enum SendResult { UNCONFIGURED_GROUPID_OR_STREAMID, // DataProxyErrCode(113) TOPIC_IS_BLANK, // DataProxyErrCode(115) DATAPROXY_FAIL_TO_RECEIVE, // DataProxyErrCode(114,116,117,118,119,120) + MESSAGE_TOO_LARGE, + WRITE_OVER_WATERMARK, /* error when water overflow */ + MAX_FLIGHT_ON_ALL_CONNECTION, + NO_REMOTE_NODE_META_INFOS, + EMPTY_ACTIVE_NODE_SET, + NO_VALID_REMOTE_NODE, + SENDER_CLOSED, UNKOWN_ERROR } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/EncryptConfigEntry.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/EncryptConfigEntry.java index 6acfe09d8ab..47f6cd1ed73 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/EncryptConfigEntry.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/EncryptConfigEntry.java @@ -25,6 +25,7 @@ import java.net.URLEncoder; import java.security.interfaces.RSAPublicKey; +import java.util.Objects; import java.util.concurrent.atomic.AtomicLong; public class EncryptConfigEntry implements java.io.Serializable { @@ -122,7 +123,7 @@ public EncryptInfo getRsaEncryptInfo() { @Override public boolean equals(Object other) { - if (other == null || !(other instanceof EncryptConfigEntry)) { + if (!(other instanceof EncryptConfigEntry)) { return false; } if (other == this) { @@ -131,7 +132,7 @@ public boolean equals(Object other) { EncryptConfigEntry info = (EncryptConfigEntry) other; return (this.userName.equals(info.getUserName())) && (this.version.equals(info.getVersion())) - && (this.pubKey == info.getPubKey()); + && (Objects.equals(this.pubKey, info.getPubKey())); } public String toString() { diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/HostInfo.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/HostInfo.java index c93c872ea55..071cf4b6eb9 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/HostInfo.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/HostInfo.java @@ -25,10 +25,10 @@ public class HostInfo implements Comparable, java.io.Serializable { private final String hostName; private final int portNumber; - public HostInfo(String referenceName, String hostName, int portNumber) { - this.referenceName = referenceName; + public HostInfo(String hostName, int portNumber) { this.hostName = hostName; this.portNumber = portNumber; + this.referenceName = hostName + ":" + portNumber; } public String getReferenceName() { diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/ProxyConfigEntry.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/ProxyConfigEntry.java index 441c62abe50..e37b9b2c0ad 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/ProxyConfigEntry.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/ProxyConfigEntry.java @@ -17,13 +17,14 @@ package org.apache.inlong.sdk.dataproxy.config; +import org.apache.commons.lang3.builder.ToStringBuilder; + import java.util.Map; public class ProxyConfigEntry implements java.io.Serializable { private int clusterId; private String groupId; - private int size; private Map hostMap; private int load; private int switchStat; @@ -59,16 +60,14 @@ public Map getHostMap() { } public void setHostMap(Map hostMap) { - this.size = hostMap.size(); this.hostMap = hostMap; } - - public int getSize() { - return size; + public boolean isNodesEmpty() { + return this.hostMap.isEmpty(); } - public void setSize(int size) { - this.size = size; + public int getSize() { + return hostMap.size(); } public String getGroupId() { @@ -87,13 +86,6 @@ public void setInterVisit(boolean interVisit) { isInterVisit = interVisit; } - @Override - public String toString() { - return "ProxyConfigEntry [hostMap=" + hostMap + ", load=" + load + ", size=" + size + ", isInterVisit=" - + isInterVisit + ", groupId=" + groupId + ", switch=" + switchStat + ", maxPacketLength=" - + maxPacketLength + "]"; - } - public int getClusterId() { return clusterId; } @@ -101,4 +93,17 @@ public int getClusterId() { public void setClusterId(int clusterId) { this.clusterId = clusterId; } + + @Override + public String toString() { + return new ToStringBuilder(this) + .append("clusterId", clusterId) + .append("groupId", groupId) + .append("hostMap", hostMap) + .append("load", load) + .append("switchStat", switchStat) + .append("isInterVisit", isInterVisit) + .append("maxPacketLength", maxPacketLength) + .toString(); + } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/ProxyConfigManager.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/ProxyConfigManager.java index 13ef45479b1..033395667b0 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/ProxyConfigManager.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/config/ProxyConfigManager.java @@ -21,21 +21,18 @@ import org.apache.inlong.common.pojo.dataproxy.DataProxyNodeResponse; import org.apache.inlong.common.util.BasicAuth; import org.apache.inlong.sdk.dataproxy.ConfigConstants; -import org.apache.inlong.sdk.dataproxy.LoadBalance; import org.apache.inlong.sdk.dataproxy.ProxyClientConfig; import org.apache.inlong.sdk.dataproxy.network.ClientMgr; -import org.apache.inlong.sdk.dataproxy.network.HashRing; -import org.apache.inlong.sdk.dataproxy.network.Utils; +import org.apache.inlong.sdk.dataproxy.network.IpUtils; +import org.apache.inlong.sdk.dataproxy.utils.LogCounter; +import org.apache.inlong.sdk.dataproxy.utils.Tuple2; import com.google.gson.Gson; -import com.google.gson.JsonArray; import com.google.gson.JsonObject; import com.google.gson.JsonParser; import com.google.gson.stream.JsonReader; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.collections.CollectionUtils; -import org.apache.commons.collections.MapUtils; -import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.ObjectUtils; import org.apache.commons.lang3.StringUtils; import org.apache.http.Header; @@ -45,7 +42,6 @@ import org.apache.http.client.entity.UrlEncodedFormEntity; import org.apache.http.client.methods.HttpPost; import org.apache.http.conn.ssl.SSLConnectionSocketFactory; -import org.apache.http.entity.StringEntity; import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.DefaultHttpClient; import org.apache.http.impl.client.HttpClients; @@ -68,7 +64,7 @@ import java.io.FileWriter; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; -import java.io.UnsupportedEncodingException; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; import java.security.KeyManagementException; @@ -78,8 +74,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; /** @@ -90,164 +87,199 @@ */ public class ProxyConfigManager extends Thread { - public static final String APPLICATION_JSON = "application/json"; - private static final Logger LOGGER = LoggerFactory.getLogger(ProxyConfigManager.class); - private final ProxyClientConfig clientConfig; - private final String localIP; - private final ClientMgr clientManager; - private final ReentrantReadWriteLock rw = new ReentrantReadWriteLock(); - private final JsonParser jsonParser = new JsonParser(); + private static final Logger logger = LoggerFactory.getLogger(ProxyConfigManager.class); + private static final LogCounter exptCounter = new LogCounter(10, 100000, 60 * 1000L); + private static final LogCounter parseCounter = new LogCounter(10, 100000, 60 * 1000L); + private static final ReentrantReadWriteLock fileRw = new ReentrantReadWriteLock(); + + private final String callerId; + private ProxyClientConfig clientConfig; private final Gson gson = new Gson(); - private final HashRing hashRing = HashRing.getInstance(); - private List proxyInfoList = new ArrayList(); - /* the status of the cluster.if this value is changed,we need rechoose three proxy */ + private final ClientMgr clientManager; + private final ThreadLocalRandom random = ThreadLocalRandom.current(); + private final AtomicBoolean shutDown = new AtomicBoolean(false); + // proxy configure info + private String localProxyConfigStoreFile; + private String proxyConfigVisitUrl; + private String proxyConfigCacheFile; + private List proxyInfoList = new ArrayList<>(); private int oldStat = 0; - private String inlongGroupId; private String localMd5; - private boolean bShutDown = false; - private long doworkTime = 0; - private EncryptConfigEntry userEncryConfigEntry; - - public ProxyConfigManager(final ProxyClientConfig configure, final String localIP, final ClientMgr clientManager) { - this.clientConfig = configure; - this.localIP = localIP; - this.clientManager = clientManager; - this.hashRing.setVirtualNode(configure.getVirtualNode()); + private long lstUpdateTime = 0; + // encrypt configure info + private String encryptConfigVisitUrl; + private String encryptConfigCacheFile; + private EncryptConfigEntry userEncryptConfigEntry; + + public ProxyConfigManager(ProxyClientConfig configure) { + this("MetaQuery", configure, null); } - public String getInlongGroupId() { - return inlongGroupId; + public ProxyConfigManager(String callerId, ProxyClientConfig configure, ClientMgr clientManager) { + this.callerId = callerId; + this.clientManager = clientManager; + this.storeAndBuildMetaConfigure(configure); + if (this.clientManager != null) { + this.setName("ConfigManager-" + this.callerId); + logger.info("ConfigManager({}) started, groupId={}", + this.callerId, clientConfig.getInlongGroupId()); + } } - public void setInlongGroupId(String inlongGroupId) { - this.inlongGroupId = inlongGroupId; + /** + * Update proxy client configure for query case + * + * @param configure proxy client configure + * @throws Exception exception + */ + public void updProxyClientConfig(ProxyClientConfig configure) throws Exception { + if (configure == null) { + throw new Exception("ProxyClientConfig is null"); + } + if (this.clientManager != null) { + throw new Exception("Not allowed for non meta-query case!"); + } + if (shutDown.get()) { + return; + } + this.storeAndBuildMetaConfigure(configure); } public void shutDown() { - LOGGER.info("Begin to shut down ProxyConfigManager!"); - bShutDown = true; - } - - @Override - public void run() { - while (!bShutDown) { - try { - doProxyEntryQueryWork(); - updateEncryptConfigEntry(); - LOGGER.info("ProxyConf update!"); - } catch (Throwable e) { - LOGGER.error("Refresh proxy ip list runs into exception {}, {}", e.toString(), e.getStackTrace()); - e.printStackTrace(); - } - - /* Sleep some time.240-360s */ - try { - Random random = new Random(); - int proxyUpdateIntervalSec = this.clientConfig.getProxyUpdateIntervalMinutes() * 60; - - int sleepTimeSec = proxyUpdateIntervalSec; - if (proxyUpdateIntervalSec > 5) { - sleepTimeSec = proxyUpdateIntervalSec + random.nextInt() % (proxyUpdateIntervalSec / 5); - } - LOGGER.info("sleep time {}", sleepTimeSec); - Thread.sleep(sleepTimeSec * 1000); - } catch (Throwable e2) { - // - } + if (clientManager == null) { + return; + } + if (shutDown.compareAndSet(false, true)) { + this.interrupt(); + logger.info("ConfigManager({}) begin to shutdown, groupId={}!", + this.callerId, clientConfig.getInlongGroupId()); } - LOGGER.info("ProxyConfigManager worker existed!"); } /** - * try to read cache of proxy entry + * get groupId config * - * @return + * @return proxyConfigEntry + * @throws Exception ex */ - private ProxyConfigEntry tryToReadCacheProxyEntry(String configCachePath) { - rw.readLock().lock(); - try { - File file = new File(configCachePath); - long diffTime = System.currentTimeMillis() - file.lastModified(); - - if (diffTime < clientConfig.getMaxProxyCacheTimeInMs()) { - JsonReader reader = new JsonReader(new FileReader(configCachePath)); - ProxyConfigEntry proxyConfigEntry = gson.fromJson(reader, ProxyConfigEntry.class); - LOGGER.info("{} has a backup! {}", inlongGroupId, proxyConfigEntry); - return proxyConfigEntry; - } - } catch (Exception ex) { - LOGGER.warn("try to read local cache, caught {}", ex.getMessage()); - } finally { - rw.readLock().unlock(); + public Tuple2 getGroupIdConfigure(boolean needRetry) throws Exception { + if (shutDown.get()) { + return new Tuple2<>(null, "SDK has shutdown!"); } - return null; - } - - private void tryToWriteCacheProxyEntry(ProxyConfigEntry entry, String configCachePath) { - rw.writeLock().lock(); - try { - File file = new File(configCachePath); - if (!file.getParentFile().exists()) { - // try to create parent - file.getParentFile().mkdirs(); + if (clientConfig.isOnlyUseLocalProxyConfig()) { + return getLocalProxyListFromFile(this.localProxyConfigStoreFile); + } else { + boolean readFromRmt = false; + Tuple2 result; + result = tryToReadCacheProxyEntry(); + if (result.getF0() == null) { + int retryCount = 0; + do { + result = requestProxyEntryQuietly(); + if (result.getF0() != null || !needRetry || shutDown.get()) { + if (result.getF0() != null) { + readFromRmt = true; + } + break; + } + // sleep then retry + TimeUnit.MILLISECONDS.sleep(500); + } while (++retryCount < clientConfig.getConfigSyncMaxRetryIfFail()); } - LOGGER.info("try to write {}} to local cache {}", entry, configCachePath); - FileWriter fileWriter = new FileWriter(configCachePath); - gson.toJson(entry, fileWriter); - fileWriter.flush(); - fileWriter.close(); - } catch (Exception ex) { - LOGGER.warn("try to write local cache, caught {}", ex.getMessage()); - } finally { - rw.writeLock().unlock(); - } - } - - private ProxyConfigEntry requestProxyEntryQuietly() { - try { - return requestProxyList(this.clientConfig.getManagerUrl()); - } catch (Exception e) { - LOGGER.warn("try to request proxy list by http, caught {}", e.getMessage()); + if (shutDown.get()) { + return new Tuple2<>(null, "SDK has shutdown!"); + } + if (result.getF0() == null) { + return new Tuple2<>(null, "Visit manager error:" + result.getF1()); + } else if (readFromRmt) { + tryToWriteCacheProxyEntry(result.getF0()); + } + return result; } - return null; } /** - * get groupId config + * get encrypt config * * @return proxyConfigEntry - * @throws Exception + * @throws Exception ex */ - public ProxyConfigEntry getGroupIdConfigure() throws Exception { - ProxyConfigEntry proxyEntry; - String configAddr = clientConfig.getConfStoreBasePath() + inlongGroupId; - if (this.clientConfig.isReadProxyIPFromLocal()) { - configAddr = configAddr + ".local"; - proxyEntry = getLocalProxyListFromFile(configAddr); - } else { - configAddr = configAddr + ".proxyip"; - - proxyEntry = tryToReadCacheProxyEntry(configAddr); - if (proxyEntry == null) { - proxyEntry = requestProxyEntryQuietly(); - int requestCount = 0; - - while (requestCount < 3 && proxyEntry == null) { - proxyEntry = requestProxyEntryQuietly(); - requestCount += 1; - if (proxyEntry == null) { - // sleep then retry - TimeUnit.MILLISECONDS.sleep(500); + public Tuple2 getEncryptConfigure(boolean needRetry) throws Exception { + if (!clientConfig.isNeedDataEncry()) { + return new Tuple2<>(null, "Not need data encrypt!"); + } + if (shutDown.get()) { + return new Tuple2<>(null, "SDK has shutdown!"); + } + EncryptConfigEntry encryptEntry = this.userEncryptConfigEntry; + if (encryptEntry != null) { + return new Tuple2<>(encryptEntry, "Ok"); + } + boolean readFromRmt = false; + Tuple2 result = readCachedPubKeyEntry(); + if (result.getF0() == null) { + int retryCount = 0; + do { + result = requestPubKeyFromManager(); + if (result.getF0() != null || !needRetry || shutDown.get()) { + if (result.getF0() != null) { + readFromRmt = true; } + break; + } + // sleep then retry + TimeUnit.MILLISECONDS.sleep(500); + } while (++retryCount < clientConfig.getConfigSyncMaxRetryIfFail()); + } + if (shutDown.get()) { + return new Tuple2<>(null, "SDK has shutdown!"); + } + if (result.getF0() == null) { + return new Tuple2<>(null, "Visit manager error:" + result.getF1()); + } else if (readFromRmt) { + updateEncryptConfigEntry(result.getF0()); + writeCachePubKeyEntryFile(result.getF0()); + } + return result; + } + + @Override + public void run() { + logger.info("ConfigManager({}) thread start, groupId={}", + this.callerId, clientConfig.getInlongGroupId()); + while (!shutDown.get()) { + // update proxy nodes meta configures + try { + doProxyEntryQueryWork(); + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) refresh proxy configure exception, groupId={}", + this.callerId, clientConfig.getInlongGroupId(), ex); } } - if (proxyEntry == null) { - throw new Exception("Visit manager error, please check log!"); - } else { - tryToWriteCacheProxyEntry(proxyEntry, configAddr); + // update encrypt configure + if (clientConfig.isNeedDataEncry()) { + try { + doEncryptConfigEntryQueryWork(); + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) refresh encrypt info exception, groupId={}", + this.callerId, clientConfig.getInlongGroupId(), ex); + } + } + } + if (shutDown.get()) { + break; + } + // sleep some time + try { + Thread.sleep(clientConfig.getManagerConfigSyncInrMs() + random.nextInt(100) * 100); + } catch (Throwable e2) { + // } } - return proxyEntry; + logger.info("ConfigManager({}) worker existed, groupId={}", + this.callerId, this.clientConfig.getInlongGroupId()); } /** @@ -256,55 +288,140 @@ public ProxyConfigEntry getGroupIdConfigure() throws Exception { * @throws Exception */ public void doProxyEntryQueryWork() throws Exception { + if (shutDown.get()) { + return; + } /* Request the configuration from manager. */ if (localMd5 == null) { localMd5 = calcHostInfoMd5(proxyInfoList); } - ProxyConfigEntry proxyEntry = null; - String configAddr = clientConfig.getConfStoreBasePath() + inlongGroupId; - if (clientConfig.isReadProxyIPFromLocal()) { - configAddr = configAddr + ".local"; - proxyEntry = getLocalProxyListFromFile(configAddr); + Tuple2 result; + if (clientConfig.isOnlyUseLocalProxyConfig()) { + result = getLocalProxyListFromFile(this.localProxyConfigStoreFile); } else { - /* Do a compare and see if it needs to re-choose the channel. */ - configAddr = configAddr + ".managerip"; - int retryCount = 1; - while (proxyEntry == null && retryCount < this.clientConfig.getProxyUpdateMaxRetry()) { - proxyEntry = requestProxyEntryQuietly(); - retryCount++; - if (proxyEntry == null) { - // sleep then retry. - TimeUnit.SECONDS.sleep(1); + int retryCnt = 0; + do { + result = requestProxyEntryQuietly(); + if (result.getF0() != null || shutDown.get()) { + break; } + // sleep then retry. + TimeUnit.SECONDS.sleep(2); + } while (++retryCnt < this.clientConfig.getConfigSyncMaxRetryIfFail() && !shutDown.get()); + if (shutDown.get()) { + return; + } + if (result.getF0() != null) { + tryToWriteCacheProxyEntry(result.getF0()); + } + /* We should exit if no local IP list and can't request it from TDManager. */ + if (localMd5 == null && result.getF0() == null) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) connect manager({}) failure, get cached configure, groupId={}", + this.callerId, this.proxyConfigVisitUrl, this.clientConfig.getInlongGroupId()); + } + result = tryToReadCacheProxyEntry(); } - if (proxyEntry != null) { - tryToWriteCacheProxyEntry(proxyEntry, configAddr); - } - /* We should exit if no local IP list and can't request it from manager. */ - if (localMd5 == null && proxyEntry == null) { - LOGGER.error("Can't connect manager at the start of proxy API {}", - this.clientConfig.getManagerUrl()); - proxyEntry = tryToReadCacheProxyEntry(configAddr); - } - if (localMd5 != null && proxyEntry == null && proxyInfoList != null) { - StringBuffer s = new StringBuffer(); - for (HostInfo tmp : proxyInfoList) { - s.append(tmp.getHostName()).append(";").append(tmp.getPortNumber()) - .append(","); + if (localMd5 != null && result.getF0() == null && proxyInfoList != null) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) connect manager({}) failure, using the last configure, groupId={}", + this.callerId, this.proxyConfigVisitUrl, this.clientConfig.getInlongGroupId()); } - LOGGER.warn("Backup proxyEntry [{}]", s); } } - if (localMd5 == null && proxyEntry == null && proxyInfoList == null) { - if (clientConfig.isReadProxyIPFromLocal()) { - throw new Exception("Local proxy address configure " - + "read failure, please check first!"); + if (localMd5 == null && result.getF0() == null && proxyInfoList == null) { + if (clientConfig.isOnlyUseLocalProxyConfig()) { + throw new Exception("Read local proxy configure failure, please check first!"); } else { throw new Exception("Connect Manager failure, please check first!"); } } - compareProxyList(proxyEntry); + compareAndUpdateProxyList(result.getF0()); + } + private void doEncryptConfigEntryQueryWork() throws Exception { + if (shutDown.get()) { + return; + } + int retryCount = 0; + Tuple2 result; + do { + result = requestPubKeyFromManager(); + if (result.getF0() != null || shutDown.get()) { + break; + } + // sleep then retry + TimeUnit.MILLISECONDS.sleep(500); + } while (++retryCount < clientConfig.getConfigSyncMaxRetryIfFail()); + if (shutDown.get()) { + return; + } + if (result.getF0() == null) { + if (this.userEncryptConfigEntry != null) { + logger.warn("ConfigManager({}) connect manager({}) failure, using the last pubKey, secretId={}", + this.callerId, this.encryptConfigVisitUrl, this.clientConfig.getAuthSecretId()); + return; + } + throw new Exception("Visit manager error:" + result.getF1()); + } + updateEncryptConfigEntry(result.getF0()); + writeCachePubKeyEntryFile(result.getF0()); + } + + public Tuple2 getLocalProxyListFromFile(String filePath) { + String strRet; + try { + byte[] fileBytes = Files.readAllBytes(Paths.get(filePath)); + strRet = new String(fileBytes); + } catch (Throwable ex) { + return new Tuple2<>(null, "Read local configure failure from " + + filePath + ", reason is " + ex.getMessage()); + } + if (StringUtils.isBlank(strRet)) { + return new Tuple2<>(null, "Blank configure local file from " + filePath); + } + return getProxyConfigEntry(strRet); + } + + private Tuple2 requestProxyEntryQuietly() { + List params = buildProxyNodeQueryParams(); + // request meta info from manager + logger.debug("ConfigManager({}) request configure to manager({}), param={}", + this.callerId, this.proxyConfigVisitUrl, params); + Tuple2 queryResult = requestConfiguration(this.proxyConfigVisitUrl, params); + if (!queryResult.getF0()) { + return new Tuple2<>(null, queryResult.getF1()); + } + // parse result + logger.debug("ConfigManager({}) received configure, from manager({}), groupId={}, result={}", + callerId, proxyConfigVisitUrl, clientConfig.getInlongGroupId(), queryResult.getF1()); + try { + return getProxyConfigEntry(queryResult.getF1()); + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) parse failure, from manager({}), groupId={}, result={}", + callerId, proxyConfigVisitUrl, clientConfig.getInlongGroupId(), queryResult.getF1(), ex); + } + return new Tuple2<>(null, ex.getMessage()); + } + } + + private String calcHostInfoMd5(List hostInfoList) { + if (hostInfoList == null || hostInfoList.isEmpty()) { + return null; + } + Collections.sort(hostInfoList); + StringBuilder hostInfoMd5 = new StringBuilder(); + for (HostInfo hostInfo : hostInfoList) { + if (hostInfo == null) { + continue; + } + hostInfoMd5.append(hostInfo.getHostName()); + hostInfoMd5.append(":"); + hostInfoMd5.append(hostInfo.getPortNumber()); + hostInfoMd5.append(";"); + } + return DigestUtils.md5Hex(hostInfoMd5.toString()); } /** @@ -312,139 +429,204 @@ public void doProxyEntryQueryWork() throws Exception { * * @param proxyEntry */ - private void compareProxyList(ProxyConfigEntry proxyEntry) { - if (proxyEntry != null) { - LOGGER.info("{}", proxyEntry.toString()); - if (proxyEntry.getSize() != 0) { - /* Initialize the current proxy information list first. */ - clientManager.setLoadThreshold(proxyEntry.getLoad()); - - List newProxyInfoList = new ArrayList(); - for (Map.Entry entry : proxyEntry.getHostMap().entrySet()) { - newProxyInfoList.add(entry.getValue()); - } + private void compareAndUpdateProxyList(ProxyConfigEntry proxyEntry) { + if ((proxyEntry == null || proxyEntry.isNodesEmpty()) + && (proxyInfoList.isEmpty() + || (System.currentTimeMillis() - lstUpdateTime) < clientConfig.getForceReChooseInrMs())) { + return; + } + int newSwitchStat; + List newBusInfoList; + if (proxyEntry == null || proxyEntry.isNodesEmpty()) { + newSwitchStat = oldStat; + newBusInfoList = new ArrayList<>(proxyInfoList.size()); + newBusInfoList.addAll(proxyInfoList); + } else { + newSwitchStat = proxyEntry.getSwitchStat(); + newBusInfoList = new ArrayList<>(proxyEntry.getSize()); + for (Map.Entry entry : proxyEntry.getHostMap().entrySet()) { + newBusInfoList.add(entry.getValue()); + } + } + String newMd5 = calcHostInfoMd5(newBusInfoList); + String oldMd5 = calcHostInfoMd5(proxyInfoList); + boolean nodeChanged = newMd5 != null && !newMd5.equals(oldMd5); + if (nodeChanged || newSwitchStat != oldStat + || (System.currentTimeMillis() - lstUpdateTime) >= clientConfig.getForceReChooseInrMs()) { + proxyInfoList = newBusInfoList; + clientManager.updateProxyInfoList(nodeChanged, proxyInfoList); + lstUpdateTime = System.currentTimeMillis(); + oldStat = newSwitchStat; + } + } - String newMd5 = calcHostInfoMd5(newProxyInfoList); - String oldMd5 = calcHostInfoMd5(proxyInfoList); - if (newMd5 != null && !newMd5.equals(oldMd5)) { - /* Choose random alive connections to send messages. */ - LOGGER.info("old md5 {} new md5 {}", oldMd5, newMd5); - proxyInfoList.clear(); - proxyInfoList = newProxyInfoList; - clientManager.setProxyInfoList(proxyInfoList); - doworkTime = System.currentTimeMillis(); - } else if (proxyEntry.getSwitchStat() != oldStat) { - /* judge cluster's switch state */ - oldStat = proxyEntry.getSwitchStat(); - if ((System.currentTimeMillis() - doworkTime) > 3 * 60 * 1000) { - LOGGER.info("switch the cluster!"); - proxyInfoList.clear(); - proxyInfoList = newProxyInfoList; - clientManager.setProxyInfoList(proxyInfoList); - } else { - LOGGER.info("only change oldStat "); - } - } else { - newProxyInfoList.clear(); - LOGGER.info("proxy IP list doesn't change, load {}", proxyEntry.getLoad()); - } - if (clientConfig.getLoadBalance() == LoadBalance.CONSISTENCY_HASH) { - updateHashRing(proxyInfoList); - } - } else { - LOGGER.error("proxyEntry's size is zero"); + private void tryToWriteCacheProxyEntry(ProxyConfigEntry entry) { + logger.debug("ConfigManager({}) write {} to cache file ({})", + this.callerId, entry, this.proxyConfigCacheFile); + fileRw.writeLock().lock(); + try { + File file = new File(this.proxyConfigCacheFile); + if (!file.getParentFile().exists()) { + // try to create parent + file.getParentFile().mkdirs(); } + FileWriter fileWriter = new FileWriter(this.proxyConfigCacheFile); + gson.toJson(entry, fileWriter); + fileWriter.flush(); + fileWriter.close(); + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) write cache file({}) exception, groupId={}, data={}", + this.callerId, this.clientConfig.getInlongGroupId(), + this.proxyConfigCacheFile, entry.toString(), ex); + } + } finally { + fileRw.writeLock().unlock(); } } - public EncryptConfigEntry getEncryptConfigEntry(final String userName) { - if (Utils.isBlank(userName)) { - return null; - } - EncryptConfigEntry encryptEntry = this.userEncryConfigEntry; - if (encryptEntry == null) { - int retryCount = 0; - encryptEntry = requestPubKey(this.clientConfig.getRsaPubKeyUrl(), userName, false); - while (encryptEntry == null && retryCount < this.clientConfig.getProxyUpdateMaxRetry()) { - encryptEntry = requestPubKey(this.clientConfig.getRsaPubKeyUrl(), userName, false); - retryCount++; - } - if (encryptEntry == null) { - encryptEntry = getStoredPubKeyEntry(userName); - if (encryptEntry != null) { - encryptEntry.getRsaEncryptedKey(); - synchronized (this) { - if (this.userEncryConfigEntry == null) { - this.userEncryConfigEntry = encryptEntry; - } else { - encryptEntry = this.userEncryConfigEntry; - } - } + /** + * try to read cache of proxy entry + * + * @return read result + */ + private Tuple2 tryToReadCacheProxyEntry() { + fileRw.readLock().lock(); + try { + File file = new File(this.proxyConfigCacheFile); + if (file.exists()) { + long diffTime = System.currentTimeMillis() - file.lastModified(); + if (clientConfig.getConfigCacheExpiredMs() > 0 + && diffTime < clientConfig.getConfigCacheExpiredMs()) { + JsonReader reader = new JsonReader(new FileReader(this.proxyConfigCacheFile)); + ProxyConfigEntry proxyConfigEntry = gson.fromJson(reader, ProxyConfigEntry.class); + return new Tuple2<>(proxyConfigEntry, "Ok"); } + return new Tuple2<>(null, "cache configure expired!"); } else { - synchronized (this) { - if (this.userEncryConfigEntry == null || this.userEncryConfigEntry != encryptEntry) { - storePubKeyEntry(encryptEntry); - encryptEntry.getRsaEncryptedKey(); - this.userEncryConfigEntry = encryptEntry; - } else { - encryptEntry = this.userEncryConfigEntry; - } - } + return new Tuple2<>(null, "no cache configure!"); } + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) read cache file({}) exception, groupId={}", + this.callerId, this.proxyConfigCacheFile, this.clientConfig.getInlongGroupId(), ex); + } + return new Tuple2<>(null, "read cache configure failure:" + ex.getMessage()); + } finally { + fileRw.readLock().unlock(); } - return encryptEntry; } - private void updateEncryptConfigEntry() { - if (Utils.isBlank(this.clientConfig.getUserName())) { - return; + private Tuple2 requestPubKeyFromManager() { + List params = buildPubKeyQueryParams(); + // request meta info from manager + logger.debug("ConfigManager({}) request pubkey to manager({}), param={}", + this.callerId, this.encryptConfigVisitUrl, params); + Tuple2 queryResult = requestConfiguration(this.encryptConfigVisitUrl, params); + if (!queryResult.getF0()) { + return new Tuple2<>(null, queryResult.getF1()); + } + logger.debug("ConfigManager({}) received pubkey from manager({}), result={}", + this.callerId, this.encryptConfigVisitUrl, queryResult.getF1()); + JsonObject pubKeyConf; + try { + pubKeyConf = JsonParser.parseString(queryResult.getF1()).getAsJsonObject(); + } catch (Throwable ex) { + if (parseCounter.shouldPrint()) { + logger.warn("ConfigManager({}) parse failure, secretId={}, config={}!", + this.callerId, this.clientConfig.getAuthSecretId(), queryResult.getF1()); + } + return new Tuple2<>(null, "parse pubkey failure:" + ex.getMessage()); } - int retryCount = 0; - EncryptConfigEntry encryptConfigEntry = requestPubKey(this.clientConfig.getRsaPubKeyUrl(), - this.clientConfig.getUserName(), false); - while (encryptConfigEntry == null && retryCount < this.clientConfig.getProxyUpdateMaxRetry()) { - encryptConfigEntry = requestPubKey(this.clientConfig.getRsaPubKeyUrl(), - this.clientConfig.getUserName(), false); - retryCount++; - } - if (encryptConfigEntry == null) { - return; + if (pubKeyConf == null) { + return new Tuple2<>(null, "No public key information"); + } + if (!pubKeyConf.has("resultCode")) { + if (parseCounter.shouldPrint()) { + logger.warn("ConfigManager({}) config failure: resultCode field not exist, secretId={}, config={}!", + this.callerId, this.clientConfig.getAuthSecretId(), queryResult.getF1()); + } + return new Tuple2<>(null, "resultCode field not exist"); } - synchronized (this) { - if (this.userEncryConfigEntry == null || this.userEncryConfigEntry != encryptConfigEntry) { - storePubKeyEntry(encryptConfigEntry); - encryptConfigEntry.getRsaEncryptedKey(); - this.userEncryConfigEntry = encryptConfigEntry; + int resultCode = pubKeyConf.get("resultCode").getAsInt(); + if (resultCode != 0) { + if (parseCounter.shouldPrint()) { + logger.warn("ConfigManager({}) config failure: resultCode != 0, secretId={}, config={}!", + this.callerId, this.clientConfig.getAuthSecretId(), queryResult.getF1()); + } + return new Tuple2<>(null, "resultCode != 0!"); + } + if (!pubKeyConf.has("resultData")) { + if (parseCounter.shouldPrint()) { + logger.warn("ConfigManager({}) config failure: resultData field not exist, secretId={}, config={}!", + this.callerId, this.clientConfig.getAuthSecretId(), queryResult.getF1()); } + return new Tuple2<>(null, "resultData field not exist"); } - return; + JsonObject resultData = pubKeyConf.get("resultData").getAsJsonObject(); + if (resultData != null) { + String publicKey = resultData.get("publicKey").getAsString(); + if (StringUtils.isBlank(publicKey)) { + if (parseCounter.shouldPrint()) { + logger.warn("ConfigManager({}) config failure: publicKey is blank, secretId={}, config={}!", + this.callerId, this.clientConfig.getAuthSecretId(), queryResult.getF1()); + } + return new Tuple2<>(null, "publicKey is blank!"); + } + String username = resultData.get("username").getAsString(); + if (StringUtils.isBlank(username)) { + if (parseCounter.shouldPrint()) { + logger.warn("ConfigManager({}) config failure: username is blank, secretId={}, config={}!", + this.callerId, this.clientConfig.getAuthSecretId(), queryResult.getF1()); + } + return new Tuple2<>(null, "username is blank!"); + } + String versionStr = resultData.get("version").getAsString(); + if (StringUtils.isBlank(versionStr)) { + if (parseCounter.shouldPrint()) { + logger.warn("ConfigManager({}) config failure: version is blank, secretId={}, config={}!", + this.callerId, this.clientConfig.getAuthSecretId(), queryResult.getF1()); + } + return new Tuple2<>(null, "version is blank!"); + } + return new Tuple2<>(new EncryptConfigEntry(username, versionStr, publicKey), "Ok"); + } + return new Tuple2<>(null, "resultData value is null!"); } - private EncryptConfigEntry getStoredPubKeyEntry(String userName) { - if (Utils.isBlank(userName)) { - LOGGER.warn(" userName(" + userName + ") is not available"); - return null; - } - EncryptConfigEntry entry; + private void updateEncryptConfigEntry(EncryptConfigEntry newEncryptEntry) { + newEncryptEntry.getRsaEncryptedKey(); + this.userEncryptConfigEntry = newEncryptEntry; + } + + private Tuple2 readCachedPubKeyEntry() { + ObjectInputStream is; FileInputStream fis = null; - ObjectInputStream is = null; - rw.readLock().lock(); + EncryptConfigEntry entry; + fileRw.readLock().lock(); try { - File file = new File(clientConfig.getConfStoreBasePath() + userName + ".pubKey"); + File file = new File(this.encryptConfigCacheFile); if (file.exists()) { - fis = new FileInputStream(file); - is = new ObjectInputStream(fis); - entry = (EncryptConfigEntry) is.readObject(); - // is.close(); - fis.close(); - return entry; + long diffTime = System.currentTimeMillis() - file.lastModified(); + if (clientConfig.getConfigCacheExpiredMs() > 0 + && diffTime < clientConfig.getConfigCacheExpiredMs()) { + fis = new FileInputStream(file); + is = new ObjectInputStream(fis); + entry = (EncryptConfigEntry) is.readObject(); + // is.close(); + fis.close(); + return new Tuple2<>(entry, "Ok"); + } + return new Tuple2<>(null, "cache PubKeyEntry expired!"); } else { - return null; + return new Tuple2<>(null, "no PubKeyEntry file!"); } - } catch (Throwable e1) { - LOGGER.error("Read " + userName + " stored PubKeyEntry error ", e1); - return null; + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) read({}) file exception, secretId={}", + callerId, encryptConfigCacheFile, clientConfig.getAuthSecretId(), ex); + } + return new Tuple2<>(null, "read PubKeyEntry file failure:" + ex.getMessage()); } finally { if (fis != null) { try { @@ -453,16 +635,16 @@ private EncryptConfigEntry getStoredPubKeyEntry(String userName) { // } } - rw.readLock().unlock(); + fileRw.readLock().unlock(); } } - private void storePubKeyEntry(EncryptConfigEntry entry) { + private void writeCachePubKeyEntryFile(EncryptConfigEntry entry) { + ObjectOutputStream p; FileOutputStream fos = null; - ObjectOutputStream p = null; - rw.writeLock().lock(); + fileRw.writeLock().lock(); try { - File file = new File(clientConfig.getConfStoreBasePath() + entry.getUserName() + ".pubKey"); + File file = new File(this.encryptConfigCacheFile); if (!file.getParentFile().exists()) { file.getParentFile().mkdir(); } @@ -474,9 +656,11 @@ private void storePubKeyEntry(EncryptConfigEntry entry) { p.writeObject(entry); p.flush(); // p.close(); - } catch (Throwable e) { - LOGGER.error("store EncryptConfigEntry " + entry.toString() + " exception ", e); - e.printStackTrace(); + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) write file({}) exception, secretId={}, content={}", + callerId, encryptConfigCacheFile, clientConfig.getAuthSecretId(), entry.toString(), ex); + } } finally { if (fos != null) { try { @@ -485,153 +669,192 @@ private void storePubKeyEntry(EncryptConfigEntry entry) { // } } - rw.writeLock().unlock(); + fileRw.writeLock().unlock(); } } - private String calcHostInfoMd5(List hostInfoList) { - if (hostInfoList == null || hostInfoList.isEmpty()) { - return null; - } - Collections.sort(hostInfoList); - StringBuffer hostInfoMd5 = new StringBuffer(); - for (HostInfo hostInfo : hostInfoList) { - if (hostInfo == null) { - continue; + /* Request new configurations from Manager. */ + private Tuple2 requestConfiguration(String url, List params) { + HttpParams myParams = new BasicHttpParams(); + HttpConnectionParams.setConnectionTimeout(myParams, clientConfig.getManagerConnTimeoutMs()); + HttpConnectionParams.setSoTimeout(myParams, clientConfig.getManagerSocketTimeoutMs()); + CloseableHttpClient httpClient; + // build http(s) client + try { + if (this.clientConfig.isVisitManagerByHttp()) { + httpClient = new DefaultHttpClient(myParams); + } else { + httpClient = getCloseableHttpClient(params); } - hostInfoMd5.append(hostInfo.getHostName()); - hostInfoMd5.append(";"); - hostInfoMd5.append(hostInfo.getPortNumber()); - hostInfoMd5.append(";"); - } - - return DigestUtils.md5Hex(hostInfoMd5.toString()); - } - - private EncryptConfigEntry requestPubKey(String pubKeyUrl, String userName, boolean needGet) { - if (Utils.isBlank(userName)) { - LOGGER.error("Queried userName is null!"); - return null; - } - List params = new ArrayList(); - params.add(new BasicNameValuePair("operation", "query")); - params.add(new BasicNameValuePair("username", userName)); - String returnStr = requestConfiguration(pubKeyUrl, params); - if (Utils.isBlank(returnStr)) { - LOGGER.info("No public key information returned from manager"); - return null; - } - JsonObject pubKeyConf = jsonParser.parse(returnStr).getAsJsonObject(); - if (pubKeyConf == null) { - LOGGER.info("No public key information returned from manager"); - return null; - } - if (!pubKeyConf.has("resultCode")) { - LOGGER.info("Parse pubKeyConf failure: No resultCode key information returned from manager"); - return null; - } - int resultCode = pubKeyConf.get("resultCode").getAsInt(); - if (resultCode != 0) { - LOGGER.info("query pubKeyConf failure, error code is " + resultCode + ", errInfo is " - + pubKeyConf.get("message").getAsString()); - return null; - } - if (!pubKeyConf.has("resultData")) { - LOGGER.info("Parse pubKeyConf failure: No resultData key information returned from manager"); - return null; - } - JsonObject resultData = pubKeyConf.get("resultData").getAsJsonObject(); - if (resultData != null) { - String publicKey = resultData.get("publicKey").getAsString(); - if (Utils.isBlank(publicKey)) { - return null; + } catch (Throwable eHttp) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) create Http(s) client failure, url={}, params={}", + this.callerId, url, params, eHttp); } - String username = resultData.get("username").getAsString(); - if (Utils.isBlank(username)) { - return null; + return new Tuple2<>(false, eHttp.getMessage()); + } + // post request and get response + HttpPost httpPost = null; + try { + httpPost = new HttpPost(url); + this.addAuthorizationInfo(httpPost); + UrlEncodedFormEntity urlEncodedFormEntity = + new UrlEncodedFormEntity(params, StandardCharsets.UTF_8); + httpPost.setEntity(urlEncodedFormEntity); + HttpResponse response = httpClient.execute(httpPost); + if (response.getStatusLine().getStatusCode() != HttpStatus.SC_OK) { + return new Tuple2<>(false, response.getStatusLine().getStatusCode() + + ":" + response.getStatusLine().getStatusCode()); + } + String returnStr = EntityUtils.toString(response.getEntity()); + if (StringUtils.isBlank(returnStr)) { + return new Tuple2<>(false, "query result is blank!"); + } + return new Tuple2<>(true, returnStr); + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ConfigManager({}) connect manager({}) exception, params={}", + this.callerId, url, params, ex); + } + return new Tuple2<>(false, ex.getMessage()); + } finally { + if (httpPost != null) { + httpPost.releaseConnection(); } - String versionStr = resultData.get("version").getAsString(); - if (Utils.isBlank(versionStr)) { - return null; + if (httpClient != null) { + httpClient.getConnectionManager().shutdown(); } - return new EncryptConfigEntry(username, versionStr, publicKey); } - return null; } - public ProxyConfigEntry getLocalProxyListFromFile(String filePath) throws Exception { - DataProxyNodeResponse proxyCluster; - try { - byte[] fileBytes = Files.readAllBytes(Paths.get(filePath)); - proxyCluster = gson.fromJson(new String(fileBytes), DataProxyNodeResponse.class); - } catch (Throwable e) { - throw new Exception("Read local proxyList File failure by " + filePath + ", reason is " + e.getCause()); - } - if (ObjectUtils.isEmpty(proxyCluster)) { - LOGGER.warn("no proxyCluster configure from local file"); - return null; + private CloseableHttpClient getCloseableHttpClient(List params) + throws NoSuchAlgorithmException, KeyManagementException { + CloseableHttpClient httpClient; + ArrayList
headers = new ArrayList<>(); + for (BasicNameValuePair paramItem : params) { + headers.add(new BasicHeader(paramItem.getName(), paramItem.getValue())); } + RequestConfig requestConfig = RequestConfig.custom() + .setConnectTimeout(clientConfig.getManagerConnTimeoutMs()) + .setSocketTimeout(clientConfig.getManagerSocketTimeoutMs()).build(); + SSLContext sslContext = SSLContexts.custom().build(); + SSLConnectionSocketFactory sslSf = new SSLConnectionSocketFactory(sslContext, + new String[]{clientConfig.getTlsVersion()}, null, + SSLConnectionSocketFactory.getDefaultHostnameVerifier()); + httpClient = HttpClients.custom().setDefaultHeaders(headers).setDefaultRequestConfig(requestConfig) + .setSSLSocketFactory(sslSf).build(); + return httpClient; + } - return getProxyConfigEntry(proxyCluster); + private void storeAndBuildMetaConfigure(ProxyClientConfig config) { + this.clientConfig = config; + StringBuilder strBuff = new StringBuilder(512); + this.proxyConfigVisitUrl = strBuff + .append(clientConfig.isVisitManagerByHttp() ? ConfigConstants.HTTP : ConfigConstants.HTTPS) + .append(clientConfig.getManagerIP()).append(":").append(clientConfig.getManagerPort()) + .append(ConfigConstants.MANAGER_DATAPROXY_API).append(clientConfig.getInlongGroupId()) + .toString(); + strBuff.delete(0, strBuff.length()); + this.localProxyConfigStoreFile = strBuff + .append(clientConfig.getConfigStoreBasePath()) + .append(ConfigConstants.META_STORE_SUB_DIR) + .append(clientConfig.getInlongGroupId()) + .append(ConfigConstants.LOCAL_DP_CONFIG_FILE_SUFFIX) + .toString(); + strBuff.delete(0, strBuff.length()); + this.proxyConfigCacheFile = strBuff + .append(clientConfig.getConfigStoreBasePath()) + .append(ConfigConstants.META_STORE_SUB_DIR) + .append(clientConfig.getInlongGroupId()) + .append(ConfigConstants.REMOTE_DP_CACHE_FILE_SUFFIX) + .toString(); + strBuff.delete(0, strBuff.length()); + this.encryptConfigVisitUrl = clientConfig.getRsaPubKeyUrl(); + this.encryptConfigCacheFile = strBuff + .append(clientConfig.getConfigStoreBasePath()) + .append(ConfigConstants.META_STORE_SUB_DIR) + .append(clientConfig.getAuthSecretId()) + .append(ConfigConstants.REMOTE_ENCRYPT_CACHE_FILE_SUFFIX) + .toString(); + strBuff.delete(0, strBuff.length()); } - private Map getStreamIdMap(JsonObject localProxyAddrJson) { - Map streamIdMap = new HashMap(); - if (localProxyAddrJson.has("tsn")) { - JsonArray jsonStreamId = localProxyAddrJson.getAsJsonArray("tsn"); - for (int i = 0; i < jsonStreamId.size(); i++) { - JsonObject jsonItem = jsonStreamId.get(i).getAsJsonObject(); - if (jsonItem != null && jsonItem.has("streamId") && jsonItem.has("sn")) { - streamIdMap.put(jsonItem.get("streamId").getAsString(), jsonItem.get("sn").getAsInt()); - } - } - } - return streamIdMap; + private void addAuthorizationInfo(HttpPost httpPost) { + httpPost.addHeader(BasicAuth.BASIC_AUTH_HEADER, + BasicAuth.genBasicAuthCredential(clientConfig.getAuthSecretId(), + clientConfig.getAuthSecretKey())); } - public ProxyConfigEntry requestProxyList(String url) { - ArrayList params = new ArrayList(); - params.add(new BasicNameValuePair("ip", this.localIP)); + private List buildProxyNodeQueryParams() { + ArrayList params = new ArrayList<>(); + params.add(new BasicNameValuePair("ip", IpUtils.getLocalIp())); params.add(new BasicNameValuePair("protocolType", clientConfig.getProtocolType())); - LOGGER.info("Begin to get configure from manager {}, param is {}", url, params); - - String resultStr = requestConfiguration(url, params); - ProxyClusterConfig clusterConfig = gson.fromJson(resultStr, ProxyClusterConfig.class); - if (clusterConfig == null || !clusterConfig.isSuccess() || clusterConfig.getData() == null) { - return null; - } + return params; + } - DataProxyNodeResponse proxyCluster = clusterConfig.getData(); - return getProxyConfigEntry(proxyCluster); + private List buildPubKeyQueryParams() { + List params = new ArrayList<>(); + params.add(new BasicNameValuePair("operation", "query")); + params.add(new BasicNameValuePair("username", clientConfig.getAuthSecretId())); + return params; } - private ProxyConfigEntry getProxyConfigEntry(DataProxyNodeResponse proxyCluster) { + private Tuple2 getProxyConfigEntry(String strRet) { + DataProxyNodeResponse proxyCluster; + try { + proxyCluster = gson.fromJson(strRet, DataProxyNodeResponse.class); + } catch (Throwable ex) { + if (parseCounter.shouldPrint()) { + logger.warn("ConfigManager({}) parse exception, groupId={}, config={}", + this.callerId, clientConfig.getInlongGroupId(), strRet, ex); + } + return new Tuple2<>(null, "parse failure:" + ex.getMessage()); + } + // parse nodeList List nodeList = proxyCluster.getNodeList(); if (CollectionUtils.isEmpty(nodeList)) { - LOGGER.error("dataproxy nodeList is empty in DataProxyNodeResponse!"); - return null; + return new Tuple2<>(null, "nodeList is empty!"); } - Map hostMap = formatHostInfoMap(nodeList); - if (MapUtils.isEmpty(hostMap)) { - return null; + HostInfo tmpHostInfo; + Map hostMap = new HashMap<>(); + for (DataProxyNodeInfo proxy : nodeList) { + if (ObjectUtils.isEmpty(proxy.getId()) + || StringUtils.isEmpty(proxy.getIp()) + || ObjectUtils.isEmpty(proxy.getPort()) + || proxy.getPort() < 0) { + if (exptCounter.shouldPrint()) { + logger.warn("Invalid proxy node: groupId={}, id={}, ip={}, port={}", + clientConfig.getInlongGroupId(), proxy.getId(), proxy.getIp(), proxy.getPort()); + } + continue; + } + tmpHostInfo = new HostInfo(proxy.getIp(), proxy.getPort()); + hostMap.put(tmpHostInfo.getReferenceName(), tmpHostInfo); } - + if (hostMap.isEmpty()) { + return new Tuple2<>(null, "no valid nodeList records!"); + } + // parse clusterId int clusterId = -1; if (ObjectUtils.isNotEmpty(proxyCluster.getClusterId())) { clusterId = proxyCluster.getClusterId(); } + // parse load int load = ConfigConstants.LOAD_THRESHOLD; if (ObjectUtils.isNotEmpty(proxyCluster.getLoad())) { load = proxyCluster.getLoad() > 200 ? 200 : (Math.max(proxyCluster.getLoad(), 0)); } + // parse isIntranet boolean isIntranet = true; - if (ObjectUtils.isNotEmpty(proxyCluster.getIsSwitch())) { - isIntranet = proxyCluster.getIsIntranet() == 1 ? true : false; + if (ObjectUtils.isNotEmpty(proxyCluster.getIsIntranet())) { + isIntranet = proxyCluster.getIsIntranet() == 1; } + // parse isSwitch int isSwitch = 0; if (ObjectUtils.isNotEmpty(proxyCluster.getIsSwitch())) { isSwitch = proxyCluster.getIsSwitch(); } + // build ProxyConfigEntry ProxyConfigEntry proxyEntry = new ProxyConfigEntry(); proxyEntry.setClusterId(clusterId); proxyEntry.setGroupId(clientConfig.getInlongGroupId()); @@ -639,202 +862,8 @@ private ProxyConfigEntry getProxyConfigEntry(DataProxyNodeResponse proxyCluster) proxyEntry.setHostMap(hostMap); proxyEntry.setSwitchStat(isSwitch); proxyEntry.setLoad(load); - proxyEntry.setSize(nodeList.size()); proxyEntry.setMaxPacketLength( proxyCluster.getMaxPacketLength() != null ? proxyCluster.getMaxPacketLength() : -1); - return proxyEntry; - } - - private Map formatHostInfoMap(List nodeList) { - Map hostMap = new HashMap<>(); - for (DataProxyNodeInfo proxy : nodeList) { - if (ObjectUtils.isEmpty(proxy.getId()) || StringUtils.isEmpty(proxy.getIp()) || ObjectUtils - .isEmpty(proxy.getPort()) || proxy.getPort() < 0) { - LOGGER.error("invalid proxy node, id:{}, ip:{}, port:{}", proxy.getId(), proxy.getIp(), - proxy.getPort()); - continue; - } - String refId = proxy.getIp() + ":" + proxy.getPort(); - hostMap.put(refId, new HostInfo(refId, proxy.getIp(), proxy.getPort())); - - } - if (hostMap.isEmpty()) { - LOGGER.error("Parse proxyList failure: address is empty for response from manager!"); - return null; - } - return hostMap; - } - - private String updateUrl(String url, int tryIdx, String localManagerIpList) { - if (tryIdx == 0) { - return url; - } - - int headerIdx = url.indexOf("://"); - if (headerIdx == -1) { - return null; - } - String header = ""; - header = url.substring(0, headerIdx + 3); - String tmpUrl = url.substring(headerIdx + 3); - int tailerIdx = tmpUrl.indexOf("/"); - if (tailerIdx == -1) { - return null; - } - String tailer = ""; - tailer = tmpUrl.substring(tailerIdx); - String[] managerIps = localManagerIpList.split(","); - String currentManagerIp = ""; - int idx = 1; - for (String managerIp : managerIps) { - if (idx++ == tryIdx) { - currentManagerIp = managerIp; - break; - } - } - if (!currentManagerIp.equals("")) { - return header + currentManagerIp + ":" + clientConfig.getManagerPort() + tailer; - } - return null; - } - - /* Request new configurations from Manager. */ - private String requestConfiguration(String url, List params) { - if (Utils.isBlank(url)) { - LOGGER.error("request url is null"); - return null; - } - // get local managerIpList - String localManagerIps = ""; - int tryIdx = 0; - while (true) { - HttpPost httpPost = null; - String returnStr = null; - HttpParams myParams = new BasicHttpParams(); - HttpConnectionParams.setConnectionTimeout(myParams, 10000); - HttpConnectionParams.setSoTimeout(myParams, clientConfig.getManagerSocketTimeout()); - CloseableHttpClient httpClient = null; - if (this.clientConfig.isRequestByHttp()) { - httpClient = new DefaultHttpClient(myParams); - } else { - try { - httpClient = getCloseableHttpClient(params); - } catch (Throwable eHttps) { - LOGGER.error("Create Https cliet failure, error 1 is ", eHttps); - eHttps.printStackTrace(); - return null; - } - } - - if (!clientConfig.isEnableSaveManagerVIps() && tryIdx > 0) { - return null; - } - // change url's manager host port when occur error - url = updateUrl(url, tryIdx, localManagerIps); - if (url == null) { - return null; - } - tryIdx++; - - LOGGER.info("Request url : " + url + ", localManagerIps : " + localManagerIps); - try { - httpPost = new HttpPost(url); - httpPost.addHeader(BasicAuth.BASIC_AUTH_HEADER, - BasicAuth.genBasicAuthCredential(clientConfig.getAuthSecretId(), - clientConfig.getAuthSecretKey())); - UrlEncodedFormEntity urlEncodedFormEntity = new UrlEncodedFormEntity(params, "UTF-8"); - httpPost.setEntity(urlEncodedFormEntity); - HttpResponse response = httpClient.execute(httpPost); - returnStr = EntityUtils.toString(response.getEntity()); - if (Utils.isNotBlank(returnStr) - && response.getStatusLine().getStatusCode() == HttpStatus.SC_OK) { - LOGGER.info("Get configure from manager is " + returnStr); - return returnStr; - } - - if (!clientConfig.isRequestByHttp()) { - return null; - } - } catch (Throwable e) { - LOGGER.error("Connect Manager error, message: {}, url is {}", e.getMessage(), url); - - if (!clientConfig.isRequestByHttp()) { - return null; - } - // get localManagerIps - localManagerIps = getLocalManagerIps(); - if (localManagerIps == null) { - return null; - } - } finally { - if (httpPost != null) { - httpPost.releaseConnection(); - } - if (httpClient != null) { - httpClient.getConnectionManager().shutdown(); - } - } - } - } - - private StringEntity getEntity(List params) throws UnsupportedEncodingException { - JsonObject jsonObject = new JsonObject(); - for (BasicNameValuePair pair : params) { - jsonObject.addProperty(pair.getName(), pair.getValue()); - } - StringEntity se = new StringEntity(jsonObject.toString()); - se.setContentType(APPLICATION_JSON); - return se; - } - - private CloseableHttpClient getCloseableHttpClient(List params) - throws NoSuchAlgorithmException, KeyManagementException { - CloseableHttpClient httpClient; - ArrayList
headers = new ArrayList
(); - for (BasicNameValuePair paramItem : params) { - headers.add(new BasicHeader(paramItem.getName(), paramItem.getValue())); - } - RequestConfig requestConfig = RequestConfig.custom().setConnectTimeout(10000) - .setSocketTimeout(clientConfig.getManagerSocketTimeout()).build(); - SSLContext sslContext = SSLContexts.custom().build(); - SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslContext, - new String[]{clientConfig.getTlsVersion()}, null, - SSLConnectionSocketFactory.getDefaultHostnameVerifier()); - httpClient = HttpClients.custom().setDefaultHeaders(headers).setDefaultRequestConfig(requestConfig) - .setSSLSocketFactory(sslsf).build(); - return httpClient; - } - - private String getLocalManagerIps() { - String localManagerIps; - try { - File localManagerIpsFile = new File(clientConfig.getManagerIpLocalPath()); - if (localManagerIpsFile.exists()) { - byte[] serialized; - serialized = FileUtils.readFileToByteArray(localManagerIpsFile); - if (serialized == null) { - LOGGER.error("Local managerIp file is empty, file path : " - + clientConfig.getManagerIpLocalPath()); - return null; - } - localManagerIps = new String(serialized, "UTF-8"); - } else { - if (!localManagerIpsFile.getParentFile().exists()) { - localManagerIpsFile.getParentFile().mkdirs(); - } - localManagerIps = ""; - LOGGER.error("Get local managerIpList not exist, file path : " - + clientConfig.getManagerIpLocalPath()); - } - } catch (Throwable t) { - localManagerIps = ""; - LOGGER.error("Get local managerIpList occur exception,", t); - } - return localManagerIps; - } - - public void updateHashRing(List newHosts) { - this.hashRing.updateNode(newHosts); - LOGGER.debug("update hash ring {}", hashRing.getVirtualNode2RealNode()); + return new Tuple2<>(proxyEntry, "ok"); } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/HttpClientExample.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/HttpClientExample.java index 3999390f9b3..2ba1938409f 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/HttpClientExample.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/HttpClientExample.java @@ -54,9 +54,8 @@ public static HttpProxySender getMessageSender(String localIP, String inLongMana proxyConfig = new ProxyClientConfig(localIP, requestByHttp, inLongManagerAddr, Integer.valueOf(inLongManagerPort), inlongGroupId, "admin", "inlong");// user and password of manager - proxyConfig.setInlongGroupId(inlongGroupId); - proxyConfig.setConfStoreBasePath(configBasePath); - proxyConfig.setReadProxyIPFromLocal(isReadProxyIPFromLocal); + proxyConfig.setConfigStoreBasePath(configBasePath); + proxyConfig.setOnlyUseLocalProxyConfig(isReadProxyIPFromLocal); proxyConfig.setDiscardOldMessage(true); proxyConfig.setProtocolType(ProtocolType.HTTP); sender = new HttpProxySender(proxyConfig); diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/TcpClientExample.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/TcpClientExample.java index eda90bdbca8..85012af1725 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/TcpClientExample.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/TcpClientExample.java @@ -70,9 +70,9 @@ public DefaultMessageSender getMessageSender(String localIP, String inLongManage dataProxyConfig = new ProxyClientConfig(localIP, requestByHttp, inLongManagerAddr, Integer.valueOf(inLongManagerPort), inlongGroupId, "admin", "inlong"); if (StringUtils.isNotEmpty(configBasePath)) { - dataProxyConfig.setConfStoreBasePath(configBasePath); + dataProxyConfig.setConfigStoreBasePath(configBasePath); } - dataProxyConfig.setReadProxyIPFromLocal(isReadProxyIPFromLocal); + dataProxyConfig.setOnlyUseLocalProxyConfig(isReadProxyIPFromLocal); dataProxyConfig.setProtocolType(ProtocolType.TCP); dataProxyConfig.setRequestTimeoutMs(20000L); messageSender = DefaultMessageSender.generateSenderByClusterId(dataProxyConfig); diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/UdpClientExample.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/UdpClientExample.java index 863f197353c..5a564b96cb3 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/UdpClientExample.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/example/UdpClientExample.java @@ -21,7 +21,6 @@ import org.apache.inlong.sdk.dataproxy.config.EncryptConfigEntry; import org.apache.inlong.sdk.dataproxy.config.EncryptInfo; import org.apache.inlong.sdk.dataproxy.network.SequentialID; -import org.apache.inlong.sdk.dataproxy.network.Utils; import org.apache.inlong.sdk.dataproxy.utils.EncryptUtil; import io.netty.bootstrap.Bootstrap; @@ -34,6 +33,7 @@ import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.DatagramPacket; import io.netty.channel.socket.nio.NioDatagramChannel; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.xerial.snappy.Snappy; @@ -55,7 +55,7 @@ public class UdpClientExample { private static final Logger logger = LoggerFactory.getLogger(UdpClientExample.class); - private static SequentialID idGenerator = new SequentialID(Utils.getLocalIp()); + private static SequentialID idGenerator = new SequentialID(); private static SecureRandom random = new SecureRandom(); @@ -180,7 +180,7 @@ private ByteBuf writeToBuf7(EncodeObject object) { if (object.isEncrypt()) { EncryptConfigEntry encryptEntry = object.getEncryptEntry(); if (encryptEntry != null) { - if (Utils.isNotBlank(endAttr)) { + if (StringUtils.isNotBlank(endAttr)) { endAttr = endAttr + "&"; } EncryptInfo encryptInfo = encryptEntry.getRsaEncryptInfo(); @@ -191,14 +191,14 @@ private ByteBuf writeToBuf7(EncodeObject object) { } } if (!object.isGroupIdTransfer()) { - if (Utils.isNotBlank(endAttr)) { + if (StringUtils.isNotBlank(endAttr)) { endAttr = endAttr + "&"; } endAttr = (endAttr + "groupId=" + object.getGroupId() + "&streamId=" + object.getStreamId()); } - if (Utils.isNotBlank(object.getMsgUUID())) { - if (Utils.isNotBlank(endAttr)) { + if (StringUtils.isNotBlank(object.getMsgUUID())) { + if (StringUtils.isNotBlank(endAttr)) { endAttr = endAttr + "&"; } endAttr = endAttr + "msgUUID=" + object.getMsgUUID(); diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/http/InternalHttpSender.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/http/InternalHttpSender.java index 62f1e2289d2..7c2738e37d2 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/http/InternalHttpSender.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/http/InternalHttpSender.java @@ -22,7 +22,6 @@ import org.apache.inlong.sdk.dataproxy.common.SendResult; import org.apache.inlong.sdk.dataproxy.config.HostInfo; import org.apache.inlong.sdk.dataproxy.network.HttpMessage; -import org.apache.inlong.sdk.dataproxy.network.Utils; import org.apache.inlong.sdk.dataproxy.utils.ConcurrentHashSet; import com.google.gson.JsonElement; @@ -179,7 +178,7 @@ private SendResult sendByHttp(List bodies, String groupId, String stream String returnStr = EntityUtils.toString(response.getEntity()); int returnCode = response.getStatusLine().getStatusCode(); - if (Utils.isBlank(returnStr) || HttpStatus.SC_OK != returnCode) { + if (StringUtils.isBlank(returnStr) || HttpStatus.SC_OK != returnCode) { throw new Exception("get config from manager failed, result: " + returnStr + ", code: " + returnCode); } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/ClientHandler.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/ClientHandler.java index 84cd9f17a03..4ded287c8c9 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/ClientHandler.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/ClientHandler.java @@ -18,6 +18,7 @@ package org.apache.inlong.sdk.dataproxy.network; import org.apache.inlong.sdk.dataproxy.codec.EncodeObject; +import org.apache.inlong.sdk.dataproxy.utils.LogCounter; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.SimpleChannelInboundHandler; @@ -26,8 +27,9 @@ public class ClientHandler extends SimpleChannelInboundHandler { - private static final Logger logger = LoggerFactory - .getLogger(ClientHandler.class); + private static final Logger logger = LoggerFactory.getLogger(ClientHandler.class); + private static final LogCounter exceptCnt = new LogCounter(10, 100000, 60 * 1000L); + private static final LogCounter thrownCnt = new LogCounter(10, 100000, 60 * 1000L); private final Sender sender; private final ClientMgr clientMgr; @@ -38,50 +40,64 @@ public ClientHandler(Sender sender, ClientMgr clientMgr) { } @Override - public void channelRead0(ChannelHandlerContext ctx, EncodeObject e) { + public void channelRead0(ChannelHandlerContext ctx, EncodeObject encodeObject) { try { - EncodeObject encodeObject = e; - logger.debug("Channel = {} , msgType = {}", ctx.channel(), encodeObject.getMsgtype()); if (encodeObject.getMsgtype() != 8) { sender.notifyFeedback(ctx.channel(), encodeObject); - } else { - clientMgr.notifyHBAck(ctx.channel(), encodeObject.getLoad()); } - } catch (Exception ex) { - logger.error("error :", ex); + } catch (Throwable ex) { + if (thrownCnt.shouldPrint()) { + logger.warn("ClientHandler({}) channelRead0 throw exception", sender.getInstanceId(), ex); + } } } @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable e) { - logger.error("this channel {} has error! , reason is {} ", ctx.channel(), e.getCause()); + if (exceptCnt.shouldPrint()) { + logger.warn("ClientHandler({})'s channel {} has error!", + sender.getInstanceId(), ctx.channel(), e); + } try { clientMgr.setConnectionFrozen(ctx.channel()); - } catch (Exception e1) { - logger.error("exceptionCaught error :", e1); + } catch (Throwable ex) { + if (thrownCnt.shouldPrint()) { + logger.warn("ClientHandler({}) exceptionCaught throw exception", + sender.getInstanceId(), ex); + } } } @Override public void channelInactive(ChannelHandlerContext ctx) throws Exception { ctx.fireChannelInactive(); - // clientMgr.resetClient(e.getChannel()); - logger.info("ClientHandler channelDisconnected {}", ctx.channel()); + if (logger.isDebugEnabled()) { + logger.debug("ClientHandler({}) channelDisconnected {}", + sender.getInstanceId(), ctx.channel()); + } try { sender.notifyConnectionDisconnected(ctx.channel()); - } catch (Exception e1) { - logger.error("exceptionCaught error {}", e1.getMessage()); + } catch (Throwable ex) { + if (thrownCnt.shouldPrint()) { + logger.warn("ClientHandler({}) channelInactive throw exception", + sender.getInstanceId(), ex); + } } } @Override public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { - // clientMgr.resetClient(e.getChannel()); - logger.info("ClientHandler channelDisconnected {}", ctx.channel()); + if (logger.isDebugEnabled()) { + logger.debug("ClientHandler({}) channelUnregistered {}", + sender.getInstanceId(), ctx.channel()); + } try { sender.notifyConnectionDisconnected(ctx.channel()); - } catch (Exception e1) { - logger.error("exceptionCaught error {}", e1.getMessage()); + } catch (Throwable ex) { + if (thrownCnt.shouldPrint()) { + logger.warn("ClientHandler({}) channelUnregistered throw exception", + sender.getInstanceId(), ex); + } } } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/ClientMgr.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/ClientMgr.java index e1412a99368..0444edbfd50 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/ClientMgr.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/ClientMgr.java @@ -17,379 +17,249 @@ package org.apache.inlong.sdk.dataproxy.network; -import org.apache.inlong.sdk.dataproxy.ConfigConstants; -import org.apache.inlong.sdk.dataproxy.LoadBalance; import org.apache.inlong.sdk.dataproxy.ProxyClientConfig; import org.apache.inlong.sdk.dataproxy.codec.EncodeObject; +import org.apache.inlong.sdk.dataproxy.common.SendResult; import org.apache.inlong.sdk.dataproxy.config.EncryptConfigEntry; import org.apache.inlong.sdk.dataproxy.config.HostInfo; import org.apache.inlong.sdk.dataproxy.config.ProxyConfigEntry; import org.apache.inlong.sdk.dataproxy.config.ProxyConfigManager; -import org.apache.inlong.sdk.dataproxy.utils.ConsistencyHashUtil; import org.apache.inlong.sdk.dataproxy.utils.EventLoopUtil; +import org.apache.inlong.sdk.dataproxy.utils.LogCounter; +import org.apache.inlong.sdk.dataproxy.utils.Tuple2; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; import io.netty.util.concurrent.DefaultThreadFactory; +import org.apache.commons.lang.mutable.MutableBoolean; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; -import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadFactory; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantReadWriteLock; public class ClientMgr { - private static final Logger logger = LoggerFactory - .getLogger(ClientMgr.class); - private static final int[] weight = { - 1, 1, 1, 1, 1, - 2, 2, 2, 2, 2, - 3, 3, 3, 3, 3, - 6, 6, 6, 6, 6, - 12, 12, 12, 12, 12, - 48, 96, 192, 384, 1000}; - private final Map clientMapData = new ConcurrentHashMap<>(); - private final ConcurrentHashMap clientMapHB = new ConcurrentHashMap<>(); - // clientMapData + clientMapHB = clientMap - private final ConcurrentHashMap clientMap = new ConcurrentHashMap<>(); - private final ConcurrentHashMap lastBadHostMap = new ConcurrentHashMap<>(); - // clientList is the valueSet of clientMapData - private final ArrayList clientList = new ArrayList<>(); - private final Map channelLoadMapData = new ConcurrentHashMap<>(); - private final Map channelLoadMapHB = new ConcurrentHashMap<>(); - /** - * Lock to protect FSNamesystem. - */ + private static final Logger logger = LoggerFactory.getLogger(ClientMgr.class); + private static final LogCounter logCounter = new LogCounter(10, 100000, 60 * 1000L); + private static final LogCounter updConExptCnt = new LogCounter(10, 100000, 60 * 1000L); + private static final LogCounter exptCounter = new LogCounter(10, 100000, 60 * 1000L); + private static final byte[] hbMsgBody = IpUtils.getLocalIp().getBytes(StandardCharsets.UTF_8); + + private final Sender sender; + private final ProxyClientConfig configure; + private final Bootstrap bootstrap; + private final ProxyConfigManager configManager; + private final SendHBThread sendHBThread; + private final AtomicBoolean started = new AtomicBoolean(false); + // proxy nodes infos private final ReentrantReadWriteLock fsLock = new ReentrantReadWriteLock(true); - private List proxyInfoList = new ArrayList<>(); - private Bootstrap bootstrap; - private int currentIndex = 0; - private ProxyClientConfig configure; - private Sender sender; - private int aliveConnections; - private int realSize; - private SendHBThread sendHBThread; - private ProxyConfigManager ipManager; - private int groupIdNum = 0; - private String groupId = ""; - private Map streamIdMap = new HashMap(); - // private static final int total_weight = 240; - private int loadThreshold; - private int loadCycle = 0; - private LoadBalance loadBalance; + private final ConcurrentHashMap proxyNodeInfos = new ConcurrentHashMap<>(); + private ConcurrentHashMap usingClientMaps = new ConcurrentHashMap<>(); + private ConcurrentHashMap deletingClientMaps = new ConcurrentHashMap<>(); + private final ConcurrentHashMap connFailNodeMap = new ConcurrentHashMap<>(); + private volatile boolean idTransNum = false; + // groupId number configure + private volatile int groupIdNum = 0; + // steamId number configures + private Map streamIdMap = new HashMap<>(); + private volatile long lastUpdateTime = -1; + private List activeNodes = new ArrayList<>(); + private final AtomicInteger reqSendIndex = new AtomicInteger(0); - public ClientMgr(ProxyClientConfig configure, Sender sender) throws Exception { + /** + * Build up the connection between the server and client. + */ + public ClientMgr(ProxyClientConfig configure, Sender sender) { this(configure, sender, null); } /** * Build up the connection between the server and client. */ - public ClientMgr(ProxyClientConfig configure, Sender sender, ThreadFactory selfDefineFactory) throws Exception { - /* Initialize the bootstrap. */ - if (selfDefineFactory == null) { - selfDefineFactory = new DefaultThreadFactory("agent-client-io", - Thread.currentThread().isDaemon()); - } - EventLoopGroup eventLoopGroup = EventLoopUtil.newEventLoopGroup(configure.getIoThreadNum(), - configure.isEnableBusyWait(), selfDefineFactory); - bootstrap = new Bootstrap(); - bootstrap.group(eventLoopGroup); - bootstrap.channel(EventLoopUtil.getClientSocketChannelClass(eventLoopGroup)); - bootstrap.option(ChannelOption.SO_RCVBUF, ConfigConstants.DEFAULT_RECEIVE_BUFFER_SIZE); - bootstrap.option(ChannelOption.SO_SNDBUF, ConfigConstants.DEFAULT_SEND_BUFFER_SIZE); - bootstrap.handler(new ClientPipelineFactory(this, sender)); - /* ready to Start the thread which refreshes the proxy list. */ - ipManager = new ProxyConfigManager(configure, Utils.getLocalIp(), this); - ipManager.setName("proxyConfigManager"); - if (configure.getInlongGroupId() != null) { - ipManager.setInlongGroupId(configure.getInlongGroupId()); - groupId = configure.getInlongGroupId(); - } - - /* - * Request the IP before starting, so that we already have three connections. - */ + public ClientMgr(ProxyClientConfig configure, Sender sender, ThreadFactory selfDefineFactory) { this.configure = configure; this.sender = sender; - this.aliveConnections = configure.getAliveConnections(); - this.loadBalance = configure.getLoadBalance(); + // Initialize the bootstrap + this.bootstrap = buildBootstrap(this.configure, this.sender, selfDefineFactory); + // initial configure manager + this.configManager = new ProxyConfigManager( + sender.getInstanceId(), configure, this); + this.sendHBThread = new SendHBThread(); + } + public void start() { + if (!started.compareAndSet(false, true)) { + return; + } try { - ipManager.doProxyEntryQueryWork(); - } catch (IOException e) { - e.printStackTrace(); - logger.info(e.getMessage()); + this.configManager.doProxyEntryQueryWork(); + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.error("ClientMgr({}) query {} exception", + sender.getInstanceId(), configure.getInlongGroupId(), ex); + } } - ipManager.setDaemon(true); - ipManager.start(); - - this.sendHBThread = new SendHBThread(); - this.sendHBThread.setName("SendHBThread"); + this.configManager.setDaemon(true); + this.configManager.start(); + // start hb thread this.sendHBThread.start(); + logger.info("ClientMgr({}) started", sender.getInstanceId()); } - public LoadBalance getLoadBalance() { - return this.loadBalance; - } - - public int getLoadThreshold() { - return loadThreshold; - } - - public void setLoadThreshold(int loadThreshold) { - this.loadThreshold = loadThreshold; - } - - public int getGroupIdNum() { - return groupIdNum; - } - - public void setGroupIdNum(int groupIdNum) { - this.groupIdNum = groupIdNum; - } - - public String getGroupId() { - return groupId; - } - - public void setGroupId(String groupId) { - this.groupId = groupId; - } - - public Map getStreamIdMap() { - return streamIdMap; - } - - public void setStreamIdMap(Map streamIdMap) { - this.streamIdMap = streamIdMap; - } - - public EncryptConfigEntry getEncryptConfigEntry() { - return this.ipManager.getEncryptConfigEntry(configure.getUserName()); - } - - public List getProxyInfoList() { - return proxyInfoList; - } - - public void setProxyInfoList(List proxyInfoList) { - try { - /* Close and remove old client. */ - writeLock(); - this.proxyInfoList = proxyInfoList; - - if (loadThreshold == 0) { - if (aliveConnections >= proxyInfoList.size()) { - realSize = proxyInfoList.size(); - aliveConnections = realSize; - logger.error("there is no enough proxy to work!"); - } else { - realSize = aliveConnections; + public void shutDown() { + if (!started.compareAndSet(true, false)) { + return; + } + this.bootstrap.config().group().shutdownGracefully(); + this.configManager.shutDown(); + this.sendHBThread.shutDown(); + if (!this.deletingClientMaps.isEmpty()) { + for (NettyClient client : this.deletingClientMaps.values()) { + if (client == null) { + continue; } - } else { - if (aliveConnections >= proxyInfoList.size()) { - realSize = proxyInfoList.size(); - aliveConnections = realSize; - logger.error("there is no idle proxy to choose for balancing!"); - } else if ((aliveConnections + 4) > proxyInfoList.size()) { - realSize = proxyInfoList.size(); - logger.warn("there is only {} idle proxy to choose for balancing!", - proxyInfoList.size() - aliveConnections); - } else { - realSize = aliveConnections + 4; + if (client.isActive()) { + this.sender.waitForAckForChannel(client.getChannel()); } + client.close(true); } - - List hostInfos = getRealHosts(proxyInfoList, realSize); - - /* Refresh the current channel connections. */ - updateAllConnection(hostInfos); - - logger.info( - "update all connection ,client map size {},client list size {}", - clientMapData.size(), clientList.size()); - - } catch (Exception e) { - logger.error(e.getMessage()); - } finally { - writeUnlock(); + this.deletingClientMaps.clear(); } + if (!this.usingClientMaps.isEmpty()) { + for (NettyClient client : this.usingClientMaps.values()) { + if (client == null) { + continue; + } + if (client.isActive()) { + this.sender.waitForAckForChannel(client.getChannel()); + } + client.close(true); + } + this.usingClientMaps.clear(); + } + this.activeNodes.clear(); + this.sender.clearCallBack(); + logger.info("ClientMgr({}) stopped!", sender.getInstanceId()); } - public int getAliveConnections() { - return aliveConnections; - } - - public void setAliveConnections(int aliveConnections) { - this.aliveConnections = aliveConnections; - } - - public void readLock() { - this.fsLock.readLock().lock(); - } - - public void readUnlock() { - this.fsLock.readLock().unlock(); - } - - public void writeLock() { - this.fsLock.writeLock().lock(); - } - - public void writeLockInterruptibly() throws InterruptedException { - this.fsLock.writeLock().lockInterruptibly(); - } - - public void writeUnlock() { - this.fsLock.writeLock().unlock(); - } - - public boolean hasWriteLock() { - return this.fsLock.isWriteLockedByCurrentThread(); - } - - public boolean hasReadLock() { - return this.fsLock.getReadHoldCount() > 0; - } - - public boolean hasReadOrWriteLock() { - return hasReadLock() || hasWriteLock(); + public ProxyConfigEntry getGroupIdConfigure() throws Exception { + if (!this.started.get()) { + throw new Exception("SDK not start or has shutdown!"); + } + Tuple2 result = + configManager.getGroupIdConfigure(true); + if (result.getF0() == null) { + throw new Exception(result.getF1()); + } + return result.getF0(); } - public ProxyConfigEntry getGroupIdConfigureInfo() throws Exception { - return ipManager.getGroupIdConfigure(); + public EncryptConfigEntry getEncryptConfigureInfo() { + if (!this.started.get()) { + return null; + } + Tuple2 result; + try { + result = configManager.getEncryptConfigure(false); + return result.getF0(); + } catch (Throwable ex) { + return null; + } } - /** - * create conn, as DataConn or HBConn - * - * @param host - * @return - */ - private boolean initConnection(HostInfo host) { - NettyClient client = clientMap.get(host); - if (client != null && client.isActive()) { - logger.info("this client {} has open!", host.getHostName()); - throw new IllegalStateException( - "The channel has already been opened"); + public Tuple2 getClientByRoundRobin(MutableBoolean allClientMaxInFlight) { + if (!this.started.get()) { + return new Tuple2<>(SendResult.SENDER_CLOSED, null); } - client = new NettyClient(bootstrap, host.getHostName(), - host.getPortNumber(), configure); - boolean bSuccess = client.connect(); - - if (clientMapData.size() < aliveConnections) { - // create data channel - if (bSuccess) { - clientMapData.put(host, client); - clientList.add(client); - clientMap.put(host, client); - logger.info("build a connection success! {},channel {}", host.getHostName(), client.getChannel()); - } else { - logger.info("build a connection fail! {}", host.getHostName()); + if (this.proxyNodeInfos.isEmpty()) { + return new Tuple2<>(SendResult.NO_REMOTE_NODE_META_INFOS, null); + } + List curNodes = this.activeNodes; + int curNodeSize = curNodes.size(); + if (curNodeSize == 0) { + return new Tuple2<>(SendResult.EMPTY_ACTIVE_NODE_SET, null); + } + String curNode; + NettyClient client; + NettyClient backClient = null; + int nullClientCnt = 0; + int incFlightFailCnt = 0; + int startPos = reqSendIndex.getAndIncrement(); + for (int step = 0; step < curNodeSize; step++) { + curNode = curNodes.get(Math.abs(startPos++) % curNodeSize); + client = usingClientMaps.get(curNode); + if (client == null) { + nullClientCnt++; + continue; } - logger.info("client map size {},client list size {}", clientMapData.size(), clientList.size()); - } else { - // data channel list is enough, create hb channel - if (bSuccess) { - clientMapHB.put(host, client); - clientMap.put(host, client); - logger.info("build a HBconnection success! {},channel {}", host.getHostName(), client.getChannel()); + if (client.isActive()) { + if (client.tryIncMsgInFlight()) { + return new Tuple2<>(SendResult.OK, client); + } else { + incFlightFailCnt++; + } } else { - logger.info("build a HBconnection fail! {}", host.getHostName()); + backClient = client; } } - return bSuccess; - } - - /** - * create conn, as DataConn or HBConn - * - * @param host - * @return - */ - private void initConnection(HostInfo host, NettyClient client) { - if (clientMapData.size() < aliveConnections) { - // create data channel - clientMapData.put(host, client); - clientList.add(client); - clientMap.put(host, client); - logger.info("build a connection success! {},channel {}", host.getHostName(), client.getChannel()); - logger.info("client map size {},client list size {}", clientMapData.size(), clientList.size()); - } else { - // data channel list is enough, create hb channel - clientMapHB.put(host, client); - clientMap.put(host, client); - logger.info("build a HBconnection success! {},channel {}", host.getHostName(), client.getChannel()); + if (nullClientCnt == curNodeSize) { + return new Tuple2<>(SendResult.EMPTY_ACTIVE_NODE_SET, null); + } else if (incFlightFailCnt + nullClientCnt == curNodeSize) { + allClientMaxInFlight.setValue(true); + return new Tuple2<>(SendResult.MAX_FLIGHT_ON_ALL_CONNECTION, null); } - } - - /** - * create conn list, as DataConn or HBConn - * - * @param host - * @return - */ - private Map initConnectionList(List host) { - Map hostInfoNettyClientMap = new HashMap<>(); - for (HostInfo hostInfo : host) { - NettyClient client = new NettyClient(bootstrap, hostInfo.getHostName(), - hostInfo.getPortNumber(), configure); - if (client.connect()) { - hostInfoNettyClientMap.put(hostInfo, client); + if (backClient != null) { + if (backClient.reconnect(false) && backClient.isActive()) { + if (backClient.tryIncMsgInFlight()) { + return new Tuple2<>(SendResult.OK, backClient); + } } } - return hostInfoNettyClientMap; + return new Tuple2<>(SendResult.NO_VALID_REMOTE_NODE, null); } - public void resetClient(Channel channel) { + public void setConnectionFrozen(Channel channel) { if (channel == null) { return; } - logger.info("reset this channel {}", channel); - for (HostInfo hostInfo : clientMap.keySet()) { - if (hostInfo == null) { + boolean found = false; + for (NettyClient client : deletingClientMaps.values()) { + if (client == null) { continue; } - NettyClient client = clientMap.get(hostInfo); - if (client != null && client.getChannel() != null + if (client.getChannel() != null && client.getChannel().id().equals(channel.id())) { - client.reconnect(); - break; + client.setFrozen(channel.id()); + logger.debug("ClientMgr({}) frozen deleting channel {}", + sender.getInstanceId(), channel); + found = true; } } - } - - public void setConnectionFrozen(Channel channel) { - if (channel == null) { + if (found) { return; } - logger.info("set this channel {} frozen", channel); - for (HostInfo hostInfo : clientMap.keySet()) { - if (hostInfo == null) { + for (NettyClient client : usingClientMaps.values()) { + if (client == null) { continue; } - NettyClient client = clientMap.get(hostInfo); - if (client != null && client.getChannel() != null + if (client.getChannel() != null && client.getChannel().id().equals(channel.id())) { - client.setFrozen(); - logger.info("end to froze this channel {}", client.getChannel().toString()); - break; + client.setFrozen(channel.id()); + if (logCounter.shouldPrint()) { + logger.info("ClientMgr({}) frozen channel {}", sender.getInstanceId(), channel); + } } } } @@ -398,773 +268,292 @@ public void setConnectionBusy(Channel channel) { if (channel == null) { return; } - logger.info("set this channel {} busy", channel); - for (HostInfo hostInfo : clientMap.keySet()) { - if (hostInfo == null) { + boolean found = false; + for (NettyClient client : deletingClientMaps.values()) { + if (client == null) { continue; } - NettyClient client = clientMap.get(hostInfo); - if (client != null && client.getChannel() != null + if (client.getChannel() != null && client.getChannel().id().equals(channel.id())) { - client.setBusy(); - break; - } - } - } - - public synchronized NettyClient getClientByRoundRobin() { - NettyClient client = null; - if (clientList.isEmpty()) { - return null; - } - int currSize = clientList.size(); - for (int retryTime = 0; retryTime < currSize; retryTime++) { - currentIndex = (++currentIndex) % currSize; - client = clientList.get(currentIndex); - if (client != null && client.isActive()) { - break; - } - } - if (client == null || !client.isActive()) { - return null; - } - // logger.info("get a client {}", client.getChannel()); - return client; - } - - public synchronized NettyClient getClientByRandom() { - NettyClient client; - if (clientList.isEmpty()) { - return null; - } - int currSize = clientList.size(); - int maxRetry = this.configure.getMaxRetry(); - Random random = new Random(System.currentTimeMillis()); - do { - int randomId = random.nextInt(); - client = clientList.get(randomId % currSize); - if (client != null && client.isActive()) { - break; + client.setBusy(channel.id()); + logger.debug("ClientMgr({}) busy deleting channel {}", sender.getInstanceId(), channel); + found = true; } - maxRetry--; - } while (maxRetry > 0); - if (client == null || !client.isActive()) { - return null; } - return client; - } - - // public synchronized NettyClient getClientByLeastConnections() {} - - public synchronized NettyClient getClientByConsistencyHash(String messageId) { - NettyClient client; - if (clientList.isEmpty()) { - return null; - } - String hash = ConsistencyHashUtil.hashMurMurHash(messageId); - HashRing cluster = HashRing.getInstance(); - HostInfo info = cluster.getNode(hash); - client = this.clientMap.get(info); - return client; - } - - public synchronized NettyClient getClientByWeightRoundRobin() { - NettyClient client = null; - double maxWeight = Double.MIN_VALUE; - int clientId = 0; - if (clientList.isEmpty()) { - return null; - } - int currSize = clientList.size(); - for (int retryTime = 0; retryTime < currSize; retryTime++) { - currentIndex = (++currentIndex) % currSize; - client = clientList.get(currentIndex); - if (client != null && client.isActive() && client.getWeight() > maxWeight) { - clientId = currentIndex; - } - } - if (client == null || !client.isActive()) { - return null; - } - return clientList.get(clientId); - } - - // public synchronized NettyClient getClientByWeightLeastConnections(){} - - public synchronized NettyClient getClientByWeightRandom() { - NettyClient client; - double maxWeight = Double.MIN_VALUE; - int clientId = 0; - if (clientList.isEmpty()) { - return null; - } - int currSize = clientList.size(); - int maxRetry = this.configure.getMaxRetry(); - Random random = new Random(System.currentTimeMillis()); - do { - int randomId = random.nextInt(); - client = clientList.get(randomId % currSize); - if (client != null && client.isActive()) { - clientId = randomId % currSize; - break; - } - maxRetry--; - } while (maxRetry > 0); - if (client == null || !client.isActive()) { - return null; - } - return clientList.get(clientId); - } - - public NettyClient getContainProxy(String proxyip) { - if (proxyip == null) { - return null; - } - for (NettyClient tmpClient : clientList) { - if (tmpClient != null && tmpClient.getServerIP() != null && tmpClient.getServerIP().equals(proxyip)) { - return tmpClient; - } - } - return null; - } - - public void shutDown() { - bootstrap.config().group().shutdownGracefully(); - - ipManager.shutDown(); - - // connectionCheckThread.shutDown(); - sendHBThread.shutDown(); - closeAllConnection(); - - } - - private void closeAllConnection() { - if (!clientMap.isEmpty()) { - logger.info("ready to close all connections!"); - for (HostInfo hostInfo : clientMap.keySet()) { - if (hostInfo == null) { - continue; - } - NettyClient client = clientMap.get(hostInfo); - if (client != null && client.isActive()) { - sender.waitForAckForChannel(client.getChannel()); - client.close(); - } - } - } - clientMap.clear(); - clientMapData.clear(); - clientMapHB.clear(); - - channelLoadMapData.clear(); - channelLoadMapHB.clear(); - clientList.clear(); - sender.clearCallBack(); - } - - private void updateAllConnection(List hostInfos) { - - List unHealthyHostList = findUnHealthyHostList(hostInfos); - List newlyAddList = findNewlyAddList(hostInfos); - Map hostInfoNettyClientMap = initConnectionList(newlyAddList); - logger.info("unhealthyHostList = {},newlyAddList = {}", unHealthyHostList, newlyAddList); - try { - writeLock(); - replaceUnHealthyHostList(unHealthyHostList, hostInfoNettyClientMap); - } catch (Exception e) { - logger.error("update Connection error", e); - } finally { - writeUnlock(); + if (found) { + return; } - - } - - private List findUnHealthyHostList(List hostInfos) { - List unHealthyHostList = new ArrayList<>(); - if (!clientMap.isEmpty()) { - for (HostInfo hostInfo : clientMap.keySet()) { - if (hostInfo == null) { - continue; - } - Optional optionalHostInfo = - hostInfos.stream().filter(hostInfo1 -> hostInfo1.equals(hostInfo)) - .findFirst(); - NettyClient client = clientMap.get(hostInfo); - if (optionalHostInfo.isPresent() && client.isActive()) { - continue; - } - unHealthyHostList.add(hostInfo); + for (NettyClient client : usingClientMaps.values()) { + if (client == null) { + continue; } - } - return unHealthyHostList; - } - - private List findNewlyAddList(List hostInfos) { - List newlyAddList = new ArrayList<>(); - if (!clientMap.isEmpty()) { - for (HostInfo hostInfo : hostInfos) { - if (hostInfo == null) { - continue; - } - Optional optionalHostInfo = - clientMap.keySet().stream().filter(hostInfo1 -> hostInfo1.equals(hostInfo)) - .findFirst(); - if (optionalHostInfo.isPresent()) { - continue; + if (client.getChannel() != null + && client.getChannel().id().equals(channel.id())) { + client.setBusy(channel.id()); + if (logCounter.shouldPrint()) { + logger.info("ClientMgr({}) busy channel {}!", sender.getInstanceId(), channel); } - newlyAddList.add(hostInfo); } } - return newlyAddList; } - private void replaceUnHealthyHostList(List unHealthyHostList, - Map hostInfoNettyClientMap) { - int index = 0; - List hostInfos = new ArrayList<>(hostInfoNettyClientMap.keySet()); - for (HostInfo unHealthyHost : unHealthyHostList) { - NettyClient client = clientMap.get(unHealthyHost); - logger.info("ready to close not in new HostInfo connections!"); - if (client != null && client.isActive()) { - sender.waitForAckForChannel(client.getChannel()); - sender.clearCallBackByChannel(client.getChannel()); - boolean close = client.close(); - clientList.remove(client); - logger.info("close connections! = {} for host = {}", close, unHealthyHost); - } - clientMap.remove(unHealthyHost); - clientMapData.remove(unHealthyHost); - clientMapHB.remove(unHealthyHost); - channelLoadMapData.remove(unHealthyHost); - channelLoadMapHB.remove(unHealthyHost); - if (index < hostInfos.size()) { - HostInfo hostInfo = hostInfos.get(index++); - initConnection(hostInfo, hostInfoNettyClientMap.get(hostInfo)); - } + public void updateProxyInfoList(boolean nodeChanged, List newNodes) { + if (newNodes == null || newNodes.isEmpty() || !this.started.get()) { + return; } - } - - public void notifyHBAck(Channel channel, short loadvalue) { + long curTime = System.currentTimeMillis(); + writeLock(); try { - if (loadvalue == (-1) || loadCycle == 0) { - return; - } else { - for (Map.Entry entry : clientMapData.entrySet()) { - NettyClient client = entry.getValue(); - HostInfo hostInfo = entry.getKey(); - if (client != null && client.getChannel() != null - && client.getChannel().id().equals(channel.id())) { - // logger.info("channel" + channel + "; Load:" + load); - if (!channelLoadMapData.containsKey(hostInfo)) { - channelLoadMapData.put(hostInfo, new int[ConfigConstants.CYCLE]); - } - if ((loadCycle - 1) >= 0) { - channelLoadMapData.get(hostInfo)[loadCycle - 1] = loadvalue; - } else { - return; + // check update info + this.updateProxyNodes(newNodes); + // shuffle candidate nodes + List candidateNodes = new ArrayList<>(this.proxyNodeInfos.size()); + candidateNodes.addAll(this.proxyNodeInfos.values()); + Collections.sort(candidateNodes); + Collections.shuffle(candidateNodes); + int curTotalCnt = candidateNodes.size(); + int needActiveCnt = Math.min(this.configure.getAliveConnections(), curTotalCnt); + // build next step nodes + NettyClient client; + int maxCycleCnt = 3; + List realHosts = new ArrayList<>(); + this.connFailNodeMap.clear(); + ConcurrentHashMap tmpClientMaps = new ConcurrentHashMap<>(); + do { + int selectCnt = 0; + for (HostInfo hostInfo : candidateNodes) { + try { + client = new NettyClient(this.sender.getInstanceId(), + this.bootstrap, hostInfo, this.configure); + if (!client.connect(true)) { + this.connFailNodeMap.put(hostInfo.getReferenceName(), hostInfo); + client.close(false); + continue; } - break; - } - } - - for (Map.Entry entry : clientMapHB.entrySet()) { - NettyClient client = entry.getValue(); - HostInfo hostInfo = entry.getKey(); - if (client != null && client.getChannel() != null - && client.getChannel().id().equals(channel.id())) { - // logger.info("HBchannel" + channel + "; Load:" + load); - if (!channelLoadMapHB.containsKey(hostInfo)) { - channelLoadMapHB.put(hostInfo, new int[ConfigConstants.CYCLE]); + realHosts.add(hostInfo.getReferenceName()); + tmpClientMaps.put(hostInfo.getReferenceName(), client); + if (++selectCnt >= needActiveCnt) { + break; } - if ((loadCycle - 1) >= 0) { - channelLoadMapHB.get(hostInfo)[loadCycle - 1] = loadvalue; - } else { - return; + } catch (Throwable ex) { + if (updConExptCnt.shouldPrint()) { + logger.warn("ClientMgr({}) build client {} exception", + this.sender.getInstanceId(), hostInfo.getReferenceName(), ex); } - break; } } - } - } catch (Exception e) { - logger.error("{} , {}", e.toString(), e.getStackTrace()); - } - } - - private void loadDataInfo(Map loadData) { - for (Map.Entry entry : channelLoadMapData.entrySet()) { - HostInfo key = entry.getKey(); - int[] value = entry.getValue(); - int numerator = 0; - int denominator = 0; - for (int i = 0; i < value.length; i++) { - if (value[i] > 0) { - numerator = numerator + value[i] * weight[i]; - denominator = denominator + weight[i]; - } - } - int sum = numerator / denominator; - loadData.put(key, sum); - } - } - - private void loadHBInfo(Map loadHB) { - for (Map.Entry entry : channelLoadMapHB.entrySet()) { - HostInfo key = entry.getKey(); - int[] value = entry.getValue(); - int numerator = 0; - int denominator = 0; - for (int i = 0; i < value.length; i++) { - if (value[i] > 0) { - numerator = numerator + value[i] * weight[i]; - denominator = denominator + weight[i]; - } - } - int sum = numerator / denominator; - loadHB.put(key, sum); - } - } - - public void notifyHBControl() { - try { - writeLock(); - logger.info("check if there is need to start balancing!"); - - Map loadData = new ConcurrentHashMap(); - Map loadHB = new ConcurrentHashMap(); - loadDataInfo(loadData); - loadHBInfo(loadHB); - - List> listData = new ArrayList<>(loadData.entrySet()); - Collections.sort(listData, new Comparator>() { - - @Override - public int compare(Map.Entry o1, Map.Entry o2) { - if (o2.getValue() != null && o1.getValue() != null && o1.getValue() > o2.getValue()) { - return -1; - } else { - return 1; - } + if (!realHosts.isEmpty()) { + break; } - }); - List> listHB = new ArrayList<>(loadHB.entrySet()); - Collections.sort(listHB, new Comparator>() { - - @Override - public int compare(Map.Entry o1, Map.Entry o2) { - if (o2.getValue() != null && o1.getValue() != null && o2.getValue() > o1.getValue()) { - return -1; - } else { - return 1; - } + Thread.sleep(1000); + } while (--maxCycleCnt > 0); + // update active nodes + if (realHosts.isEmpty()) { + if (nodeChanged) { + logger.error("ClientMgr({}) changed nodes, but all connect failure, nodes={}!", + this.sender.getInstanceId(), candidateNodes); + } else { + logger.error("ClientMgr({}) re-choose nodes, but all connect failure, nodes={}!", + this.sender.getInstanceId(), candidateNodes); } - }); - - logger.info("show info: last compute result!"); - for (Map.Entry item : listData) { - // System.out.println("listData:"+listData.get(i)); - logger.info("Client:" + item.getKey() + ";" + item.getValue()); - } - for (Map.Entry item : listHB) { - // System.out.println("listHB:"+listHB.get(i)); - logger.info("HBClient:" + item.getKey() + ";" + item.getValue()); - } - boolean isLoadSwitch = false; - int smallSize = Math.min(Math.min(listData.size(), listHB.size()), 1); - for (int i = 0; i < smallSize; i++) { - if ((listData.get(i).getValue() - listHB.get(i).getValue()) >= this.loadThreshold) { - isLoadSwitch = true; - HostInfo dataHost = listData.get(i).getKey(); - HostInfo hbHost = listHB.get(i).getKey(); - logger.info("balancing client:" + dataHost.getHostName() + ",load: " + listData.get(i).getValue() - + "; HBclient:" + hbHost.getHostName() + ",load: " + listHB.get(i).getValue()); - - NettyClient client = clientMapData.get(dataHost); - client.setFrozen(); - sender.waitForAckForChannel(client.getChannel()); - client.close(); - - clientList.remove(clientMapData.get(dataHost)); - clientMap.remove(dataHost); - clientMapData.remove(dataHost); - // channelLoadMapData.remove(dataHost); - clientMapData.put(hbHost, clientMapHB.get(hbHost)); - // channelLoadMapData.put(hbHost,listHB.get(i).getValue()); - clientList.add(clientMapHB.get(hbHost)); - clientMapHB.remove(hbHost); + } else { + this.lastUpdateTime = System.currentTimeMillis(); + this.deletingClientMaps = this.usingClientMaps; + this.usingClientMaps = tmpClientMaps; + this.activeNodes = realHosts; + if (nodeChanged) { + logger.info("ClientMgr({}) changed nodes, wast {}ms, nodeCnt=(r:{}-a:{}), actives={}, fail={}", + sender.getInstanceId(), (System.currentTimeMillis() - curTime), + needActiveCnt, realHosts.size(), realHosts, connFailNodeMap.keySet()); + } else { + logger.info("ClientMgr({}) re-choose nodes, wast {}ms, nodeCnt=(r:{}-a:{}), actives={}, fail={}", + sender.getInstanceId(), (System.currentTimeMillis() - curTime), + needActiveCnt, realHosts.size(), realHosts, connFailNodeMap.keySet()); } } - - if (!isLoadSwitch) { - logger.info("Choose other HBClient because there is no load balancing! "); - } - for (Map.Entry entry : clientMapHB.entrySet()) { - entry.getValue().close(); - clientMap.remove(entry.getKey()); + } catch (Throwable ex) { + if (updConExptCnt.shouldPrint()) { + logger.warn("ClientMgr({}) update nodes throw exception", sender.getInstanceId(), ex); } - clientMapHB.clear(); - - int realSize = this.realSize - clientMap.size(); - if (realSize > 0) { - List hostInfoList = new ArrayList<>(proxyInfoList); - hostInfoList.removeAll(clientMap.keySet()); - List replaceHost = getRealHosts(hostInfoList, realSize); - for (HostInfo hostInfo : replaceHost) { - initConnection(hostInfo); - } - } - } catch (Exception e) { - logger.error("notifyHBcontrol", e); } finally { writeUnlock(); } } - private void sendHeartBeat() { - // all hbChannels need hb - for (Map.Entry clientEntry : clientMapHB.entrySet()) { - if (clientEntry.getKey() != null && clientEntry.getValue() != null) { - sendHeartBeat(clientEntry.getKey(), clientEntry.getValue()); - } - } + public String getGroupId() { + return configure.getInlongGroupId(); + } - // only idle dataChannels need hb - for (Map.Entry clientEntry : clientMapData.entrySet()) { - if (clientEntry.getKey() == null || clientEntry.getValue() == null) { - continue; - } - if (sender.isIdleClient(clientEntry.getValue())) { - sendHeartBeat(clientEntry.getKey(), clientEntry.getValue()); - } - } + public boolean isIdTransNum() { + return idTransNum; + } + public int getGroupIdNum() { + return groupIdNum; } - private void sendHeartBeat(HostInfo hostInfo, NettyClient client) { - if (!client.isActive()) { - logger.info("client {} is inActive", hostInfo.getReferenceName()); - return; - } - logger.debug("active host to send heartbeat! {}", hostInfo.getReferenceName()); - String hbMsg = "heartbeat:" + hostInfo.getHostName(); - EncodeObject encodeObject = new EncodeObject( - Collections.singletonList(hbMsg.getBytes(StandardCharsets.UTF_8)), - 8, false, false, false, System.currentTimeMillis() / 1000, 1, "", "", ""); - try { - if (configure.isNeedAuthentication()) { - encodeObject.setAuth(configure.isNeedAuthentication(), - configure.getUserName(), configure.getSecretKey()); + public int getStreamIdNum(String streamId) { + if (idTransNum) { + Integer tmpNum = streamIdMap.get(streamId); + if (tmpNum != null) { + return tmpNum; } - client.write(encodeObject); - } catch (Throwable e) { - logger.error("sendHeartBeat to " + hostInfo.getReferenceName() - + " exception {}, {}", e.toString(), e.getStackTrace()); } + return 0; } - /** - * fill up client with hb client - */ - private void fillUpWorkClientWithHBClient() { - if (clientMapHB.size() > 0) { - logger.info("fill up work client with HB, clientMapData {}, clientMapHB {}", - clientMapData.size(), clientMapHB.size()); - } - Iterator> it = clientMapHB.entrySet().iterator(); - while (it.hasNext() && clientMapData.size() < aliveConnections) { - Map.Entry entry = it.next(); - clientMapData.put(entry.getKey(), entry.getValue()); - clientList.add(entry.getValue()); - channelLoadMapHB.remove(entry.getKey()); - it.remove(); + public void updateGroupIdAndStreamIdNumInfo( + int groupIdNum, Map streamIdMap) { + this.groupIdNum = groupIdNum; + this.streamIdMap = streamIdMap; + if (groupIdNum != 0 && streamIdMap != null && !streamIdMap.isEmpty()) { + this.idTransNum = true; } } - private void fillUpWorkClientWithLastBadClient() { - - int currentRealSize = aliveConnections - clientMapData.size(); + private void writeLock() { + this.fsLock.writeLock().lock(); + } - List pendingBadList = new ArrayList<>(); - for (Map.Entry entry : lastBadHostMap.entrySet()) { - if (pendingBadList.size() < currentRealSize) { - pendingBadList.add(entry.getKey()); - } else { - for (int index = 0; index < pendingBadList.size(); index++) { + private void writeUnlock() { + this.fsLock.writeLock().unlock(); + } - if (entry.getValue().get() < lastBadHostMap - .get(pendingBadList.get(index)).get()) { - pendingBadList.set(index, entry.getKey()); - } + private void updateProxyNodes(List newProxyNodes) { + boolean found; + List rmvNodes = new ArrayList<>(); + for (String hostRefName : this.proxyNodeInfos.keySet()) { + found = false; + for (HostInfo hostInfo : newProxyNodes) { + if (hostRefName.equals(hostInfo.getReferenceName())) { + found = true; + break; } } - } - List replaceHostLists = getRealHosts(pendingBadList, currentRealSize); - if (replaceHostLists.size() > 0) { - logger.info("replace bad connection, use last bad list, " - + "last bad list {}, client Map data {}", - lastBadHostMap.size(), clientMapData.size()); - } - for (HostInfo hostInfo : replaceHostLists) { - - boolean isSuccess = initConnection(hostInfo); - - if (isSuccess) { - lastBadHostMap.remove(hostInfo); + if (!found) { + rmvNodes.add(hostRefName); } } - } - - private void fillUpWorkClientWithBadClient(List badHostLists) { - if (badHostLists.isEmpty()) { - logger.warn("badHostLists is empty, current hostList size {}, dataClient size {}, hbClient size {}", - proxyInfoList.size(), clientMapData.size(), clientMapHB.size()); - return; - } - logger.info("all hosts are bad, dataClient is empty, reuse them, badHostLists size {}, " - + "proxyInfoList size {}", badHostLists.size(), proxyInfoList.size()); - - List replaceHostLists = getRealHosts(badHostLists, aliveConnections); - boolean isSuccess = false; - for (HostInfo hostInfo : replaceHostLists) { - isSuccess = initConnection(hostInfo); - if (isSuccess) { - badHostLists.remove(hostInfo); - } - } - } - - private void removeBadRealClient(List badHostLists, List normalHostLists) { - for (HostInfo hostInfo : clientMapData.keySet()) { - if (hostInfo == null) { + for (HostInfo hostInfo : newProxyNodes) { + if (this.proxyNodeInfos.containsKey(hostInfo.getReferenceName())) { continue; } - NettyClient client = clientMapData.get(hostInfo); - if (client == null || !client.isActive()) { - logger.info("this host {} is bad! so remove it", hostInfo.getHostName()); - badHostLists.add(hostInfo); - } else { - logger.info("this host {} is active! so keep it", hostInfo.getHostName()); - normalHostLists.add(hostInfo); - } + this.proxyNodeInfos.put(hostInfo.getReferenceName(), hostInfo); + } + for (String rmvNode : rmvNodes) { + this.proxyNodeInfos.remove(rmvNode); } } - private void removeBadHBClients(List badHostLists, List normalHostLists) { - for (HostInfo hostInfo : clientMapHB.keySet()) { - if (hostInfo == null) { - continue; - } - NettyClient client = clientMapHB.get(hostInfo); - if (client == null || !client.isActive()) { - logger.info("this HBhost {} is bad! so remove it", hostInfo.getHostName()); - badHostLists.add(hostInfo); - } else { - logger.info("this HBhost {} is active! so keep it", hostInfo.getHostName()); - normalHostLists.add(hostInfo); - } + private Bootstrap buildBootstrap(ProxyClientConfig config, Sender sender, ThreadFactory selfFactory) { + if (selfFactory == null) { + selfFactory = new DefaultThreadFactory( + "sdk-netty-workers", Thread.currentThread().isDaemon()); } + EventLoopGroup eventLoopGroup = EventLoopUtil.newEventLoopGroup( + config.getIoThreadNum(), config.isEnableBusyWait(), selfFactory); + Bootstrap tmpBootstrap = new Bootstrap(); + tmpBootstrap.group(eventLoopGroup); + tmpBootstrap.channel(EventLoopUtil.getClientSocketChannelClass(eventLoopGroup)); + tmpBootstrap.option(ChannelOption.SO_RCVBUF, config.getRecvBufferSize()); + tmpBootstrap.option(ChannelOption.SO_SNDBUF, config.getSendBufferSize()); + tmpBootstrap.handler(new ClientPipelineFactory(this, sender)); + return tmpBootstrap; } - private void removeBadClients(List badHostLists) { - for (HostInfo hostInfo : badHostLists) { - if (hostInfo == null) { - continue; - } - NettyClient client = clientMapData.get(hostInfo); - if (client != null) { - sender.waitForAckForChannel(client.getChannel()); - client.close(); - clientMapData.remove(hostInfo); - clientMap.remove(hostInfo); - clientList.remove(client); - - channelLoadMapData.remove(hostInfo); - logger.info("remove this client {}", hostInfo.getHostName()); - } - client = clientMapHB.get(hostInfo); - if (client != null) { - clientMapHB.get(hostInfo).close(); - clientMapHB.remove(hostInfo); - clientMap.remove(hostInfo); - - channelLoadMapHB.remove(hostInfo); - logger.info("remove this HBclient {}", hostInfo.getHostName()); + private void sendHeartBeatMsg(NettyClient client) { + if (!client.isActive()) { + logger.debug("ClientMgr({}) to {} is inActive", + sender.getInstanceId(), client.getNodeAddress()); + return; + } + if (!client.getChannel().isWritable()) { + if (logCounter.shouldPrint()) { + logger.warn("ClientMgr({}) to {} write_over_water", + sender.getInstanceId(), client.getChannel()); } + return; } - } - - /** - * 1. check all inactive conn, including dataConn and HBConn - * 2.1. if there is no any bad conn, do dataConn<->hbConn balance every ConfigConstants.CYCLE, according to load - * 2.2. close and remove all inactive conn - * 3. fillUp dataConn and HBConn using remaining hosts(excludes lastBadHostMap) - * 4. if dataConns are still not full, try to using HBConns (then lastBadHostMap) to fillUp - * 5. update lastBadHostMap, including increase, remove and update badValue - */ - public void replaceBadConnectionHB() { + EncodeObject encodeObject = new EncodeObject( + Collections.singletonList(hbMsgBody), + 8, false, false, false, System.currentTimeMillis() / 1000, 1, "", "", ""); try { - writeLock(); - - List badHostLists = new ArrayList<>(); - List normalHostLists = new ArrayList<>(); - removeBadRealClient(badHostLists, normalHostLists); - removeBadHBClients(badHostLists, normalHostLists); - removeBadClients(badHostLists); - - if (badHostLists.size() == 0 && normalHostLists.size() != 0 && clientMapData.size() >= aliveConnections) { - logger.info("hasn't bad host! so keep it"); - if (loadCycle >= ConfigConstants.CYCLE) { - if (loadThreshold == 0) { - logger.info("the proxy cluster is being updated!"); - } else if (clientMapHB.size() != 0 && clientMapData.size() != 0) { - notifyHBControl(); - } else if (this.realSize != clientMap.size()) { - logger.info("make the amount of proxy to original value"); - int realSize = this.realSize - clientMap.size(); - if (realSize > 0) { - List hostInfoList = new ArrayList<>(proxyInfoList); - hostInfoList.removeAll(clientMap.keySet()); - List replaceHost = getRealHosts(hostInfoList, realSize); - for (HostInfo hostInfo : replaceHost) { - initConnection(hostInfo); - } - } - } - loadCycle = 0; - channelLoadMapData.clear(); - channelLoadMapHB.clear(); - } - return; - } else { - loadCycle = 0; - channelLoadMapData.clear(); - channelLoadMapHB.clear(); - } - - List hostLists = new ArrayList(this.proxyInfoList); - hostLists.removeAll(badHostLists); - hostLists.removeAll(lastBadHostMap.keySet()); - hostLists.removeAll(normalHostLists); // now, all hosts in this hostLists are not built conn - - int realSize = this.realSize - clientMap.size(); - if (realSize > hostLists.size()) { - realSize = hostLists.size(); - } - - if (realSize != 0) { - List replaceHostLists = getRealHosts(hostLists, realSize); - /* Build new channels. */ - for (HostInfo hostInfo : replaceHostLists) { - initConnection(hostInfo); - } - } - - if (clientMapData.size() < aliveConnections) { - fillUpWorkClientWithHBClient(); - } - - if (clientMapData.size() < aliveConnections) { - fillUpWorkClientWithLastBadClient(); - } - - // when all hosts are bad, reuse them to avoid clientMapData is empty in this round - if (clientMapData.isEmpty()) { - fillUpWorkClientWithBadClient(badHostLists); - } - - for (HostInfo hostInfo : badHostLists) { - AtomicLong tmpValue = new AtomicLong(0); - AtomicLong hostValue = lastBadHostMap.putIfAbsent(hostInfo, tmpValue); - if (hostValue == null) { - hostValue = tmpValue; - } - hostValue.incrementAndGet(); + if (configure.isEnableAuthentication()) { + encodeObject.setAuth(configure.isEnableAuthentication(), + configure.getAuthSecretId(), configure.getAuthSecretKey()); } - - for (HostInfo hostInfo : normalHostLists) { - lastBadHostMap.remove(hostInfo); + client.write(encodeObject); + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ClientMgr({}) send heartBeat to {} exception ", + sender.getInstanceId(), client.getNodeAddress(), ex); } - - logger.info( - "replace bad connection ,client map size {},client list size {}", - clientMapData.size(), clientList.size()); - - } catch (Exception e) { - logger.error("replaceBadConnection exception {}, {}", e.toString(), e.getStackTrace()); - } finally { - writeUnlock(); - } - - } - - private List getRealHosts(List hostList, int realSize) { - if (realSize > hostList.size()) { - return hostList; - } - Collections.shuffle(hostList); - List resultHosts = new ArrayList(realSize); - for (int i = 0; i < realSize; i++) { - resultHosts.add(hostList.get(i)); - logger.info("host={}", hostList.get(i)); } - return resultHosts; - } - - public NettyClient getClient(LoadBalance loadBalance, EncodeObject encodeObject) { - NettyClient client = null; - switch (loadBalance) { - case RANDOM: - client = getClientByRandom(); - break; - case CONSISTENCY_HASH: - client = getClientByConsistencyHash(encodeObject.getMessageId()); - break; - case ROBIN: - client = getClientByRoundRobin(); - break; - case WEIGHT_ROBIN: - client = getClientByWeightRoundRobin(); - break; - case WEIGHT_RANDOM: - client = getClientByWeightRandom(); - break; - } - return client; } private class SendHBThread extends Thread { - private final int[] random = {17, 19, 23, 31, 37}; - private boolean bShutDown = false; + private volatile boolean bShutDown; public SendHBThread() { bShutDown = false; + this.setName("SendHBThread-" + sender.getInstanceId()); } public void shutDown() { - logger.info("begin to shut down SendHBThread!"); + logger.info("ClientMgr({}) shutdown SendHBThread!", sender.getInstanceId()); bShutDown = true; + this.interrupt(); } @Override public void run() { + logger.info("ClientMgr({}) start SendHBThread!", sender.getInstanceId()); + long curTime; while (!bShutDown) { try { - loadCycle++; - sendHeartBeat(); - replaceBadConnectionHB(); + curTime = System.currentTimeMillis(); + if (deletingClientMaps != null && !deletingClientMaps.isEmpty()) { + if (lastUpdateTime > 0 + && curTime - lastUpdateTime > configure.getConCloseWaitPeriodMs()) { + for (NettyClient client : deletingClientMaps.values()) { + if (client == null) { + continue; + } + client.close(false); + } + deletingClientMaps.clear(); + } + } + for (NettyClient nettyClient : usingClientMaps.values()) { + if (nettyClient == null) { + continue; + } + if (nettyClient.isActive()) { + if (nettyClient.isIdleClient(curTime)) { + sendHeartBeatMsg(nettyClient); + } + } else { + if (nettyClient.getMsgInFlight() <= 0L) { + nettyClient.reconnect(false); + } + } + } + if (bShutDown) { + break; + } try { - int index = (int) (Math.random() * random.length); - Thread.sleep((random[index]) * 1000); + Thread.sleep(4000L); } catch (InterruptedException e) { - // TODO Auto-generated catch block - logger.error(e.toString()); + // + } + } catch (Throwable ex) { + if (exptCounter.shouldPrint()) { + logger.warn("ClientMgr({}) SendHBThread throw exception", sender.getInstanceId(), ex); } - } catch (Throwable e) { - logger.error("SendHBThread throw exception: ", e); } } + logger.info("ClientMgr({}) exit SendHBThread!", sender.getInstanceId()); } } - } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/HashRing.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/HashRing.java deleted file mode 100644 index ea707195b14..00000000000 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/HashRing.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.inlong.sdk.dataproxy.network; - -import org.apache.inlong.sdk.dataproxy.ConfigConstants; -import org.apache.inlong.sdk.dataproxy.config.HostInfo; -import org.apache.inlong.sdk.dataproxy.utils.ConsistencyHashUtil; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.stream.Collectors; - -public class HashRing { - - private static final Logger LOGGER = LoggerFactory.getLogger(HashRing.class); - private static volatile HashRing instance; - private int virtualNode = ConfigConstants.DEFAULT_VIRTUAL_NODE; - private TreeMap virtualNode2RealNode; - private List nodeList; - - private HashRing() { - this.virtualNode2RealNode = new TreeMap<>(); - this.nodeList = new ArrayList<>(); - } - - public static HashRing getInstance() { - if (instance == null) { - synchronized (HashRing.class) { - if (instance == null) { - instance = new HashRing(); - } - } - } - return instance; - } - - public TreeMap getVirtualNode2RealNode() { - return virtualNode2RealNode; - } - - public String node2VirtualNode(HostInfo node, int index) { - return "virtual&&" + index + "&&" + node.toString(); - } - - public void setVirtualNode(int virtualNode) { - this.virtualNode = virtualNode; - } - - public HostInfo getNode(String key) { - String hash = ConsistencyHashUtil.hashMurMurHash(key); - SortedMap tailMap = this.virtualNode2RealNode.tailMap(hash); - HostInfo node; - if (tailMap.isEmpty()) { - node = this.virtualNode2RealNode.get(this.virtualNode2RealNode.firstKey()); - } else { - node = this.virtualNode2RealNode.get(tailMap.firstKey()); - } - LOGGER.info("{} located to {}", key, node); - return node; - } - - private synchronized void extendNode(List nodes) { - this.nodeList.addAll(nodes); - for (HostInfo host : this.nodeList) { - for (int i = 0; i < this.virtualNode; i++) { - String key = node2VirtualNode(host, i); - String hash = ConsistencyHashUtil.hashMurMurHash(key); - virtualNode2RealNode.put(hash, host); - } - } - LOGGER.info("append node list {}", nodes); - } - - private synchronized void removeNode(List hosts) { - for (HostInfo host : hosts) { - this.nodeList.remove(host); - for (int i = 0; i < this.virtualNode; i++) { - String hash = ConsistencyHashUtil.hashMurMurHash(node2VirtualNode(host, i)); - virtualNode2RealNode.remove(hash); - } - } - LOGGER.info("remove node list {}", hosts); - } - - public synchronized void updateNode(List nodes) { - List newHosts = new ArrayList<>(nodes); - List oldHosts = new ArrayList<>(this.nodeList); - List append = newHosts.stream().filter(host -> !oldHosts.contains(host)).collect(Collectors.toList()); - List remove = oldHosts.stream().filter(host -> !newHosts.contains(host)).collect(Collectors.toList()); - extendNode(append); - removeNode(remove); - } -} diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/HttpProxySender.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/HttpProxySender.java index 4ef884ae8fa..1cf9939ff98 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/HttpProxySender.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/HttpProxySender.java @@ -26,6 +26,7 @@ import org.apache.inlong.sdk.dataproxy.config.ProxyConfigManager; import org.apache.inlong.sdk.dataproxy.http.InternalHttpSender; import org.apache.inlong.sdk.dataproxy.utils.ConcurrentHashSet; +import org.apache.inlong.sdk.dataproxy.utils.Tuple2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,16 +76,14 @@ public HttpProxySender(ProxyClientConfig configure) throws Exception { private void initTDMClientAndRequest(ProxyClientConfig configure) throws Exception { try { - proxyConfigManager = new ProxyConfigManager(configure, - Utils.getLocalIp(), null); - proxyConfigManager.setInlongGroupId(configure.getInlongGroupId()); + proxyConfigManager = new ProxyConfigManager(configure); ProxyConfigEntry proxyConfigEntry = retryGettingProxyConfig(); hostList.addAll(proxyConfigEntry.getHostMap().values()); this.setDaemon(true); this.start(); } catch (Throwable e) { - if (configure.isReadProxyIPFromLocal()) { + if (configure.isOnlyUseLocalProxyConfig()) { throw new Exception("Get local proxy configure failure! e = {}", e); } else { throw new Exception("Visit TDManager error! e = {}", e); @@ -99,7 +98,9 @@ private void initTDMClientAndRequest(ProxyClientConfig configure) throws Excepti * @return proxy config entry. */ private ProxyConfigEntry retryGettingProxyConfig() throws Exception { - return proxyConfigManager.getGroupIdConfigure(); + Tuple2 result = + proxyConfigManager.getGroupIdConfigure(true); + return result.getF0(); } /** @@ -113,9 +114,13 @@ public void run() { int randSleepTime = proxyClientConfig.getProxyHttpUpdateIntervalMinutes() * 60 + rand; TimeUnit.MILLISECONDS.sleep(randSleepTime * 1000); if (proxyConfigManager != null) { - ProxyConfigEntry proxyConfigEntry = proxyConfigManager.getGroupIdConfigure(); - hostList.addAll(proxyConfigEntry.getHostMap().values()); - hostList.retainAll(proxyConfigEntry.getHostMap().values()); + Tuple2 result = + proxyConfigManager.getGroupIdConfigure(false); + if (result.getF0() == null) { + throw new Exception(result.getF1()); + } + hostList.addAll(result.getF0().getHostMap().values()); + hostList.retainAll(result.getF0().getHostMap().values()); } else { logger.error("manager is null, please check it!"); } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/Utils.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/IpUtils.java similarity index 91% rename from inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/Utils.java rename to inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/IpUtils.java index 3e2fec92500..f3fbab04aa2 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/Utils.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/IpUtils.java @@ -30,9 +30,9 @@ import java.util.List; import java.util.Set; -public class Utils { +public class IpUtils { - private static final Logger logger = LoggerFactory.getLogger(Utils.class); + private static final Logger logger = LoggerFactory.getLogger(IpUtils.class); private static String userIp; static { @@ -69,23 +69,6 @@ public static boolean validLocalIp(String currLocalHost) throws ProxysdkExceptio return true; } - public static boolean isNotBlank(String str) { - return !isBlank(str); - } - - public static boolean isBlank(String str) { - int strLen; - if (str == null || (strLen = str.length()) == 0) { - return true; - } - for (int i = 0; i < strLen; i++) { - if ((!Character.isWhitespace(str.charAt(i)))) { - return false; - } - } - return true; - } - public static byte[] toBytes(String ipAddr) { byte[] ret = new byte[4]; try { diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/NettyClient.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/NettyClient.java index f5297303844..bdbf12f4444 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/NettyClient.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/NettyClient.java @@ -19,106 +19,112 @@ import org.apache.inlong.sdk.dataproxy.ProxyClientConfig; import org.apache.inlong.sdk.dataproxy.codec.EncodeObject; +import org.apache.inlong.sdk.dataproxy.config.HostInfo; +import org.apache.inlong.sdk.dataproxy.utils.LogCounter; -import com.sun.management.OperatingSystemMXBean; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.lang.management.ManagementFactory; import java.net.InetSocketAddress; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; public class NettyClient { private static final Logger logger = LoggerFactory.getLogger(NettyClient.class); + private static final LogCounter conExptCnt = new LogCounter(10, 100000, 60 * 1000L); - private Channel channel = null; - private final ReentrantLock stateLock = new ReentrantLock(); - - private ConnState connState; - private ProxyClientConfig configure; - private Bootstrap bootstrap; - private String serverIP; - private int serverPort; - - public String getServerIP() { - return serverIP; - } + private final static int CLIENT_STATUS_INIT = -1; + private final static int CLIENT_STATUS_READY = 0; + private final static int CLIENT_STATUS_FROZEN = 1; + private final static int CLIENT_STATUS_DEAD = 2; + private final static int CLIENT_STATUS_BUSY = 3; - public void setServerIP(String serverIP) { - this.serverIP = serverIP; - } + private final String callerId; + private final ProxyClientConfig configure; + private final Bootstrap bootstrap; + private final HostInfo hostInfo; + private Channel channel = null; + private final AtomicInteger conStatus = new AtomicInteger(CLIENT_STATUS_INIT); + private final AtomicLong msgInFlight = new AtomicLong(0); + private final AtomicLong lstSendTime = new AtomicLong(0); + private final Semaphore reconSemaphore = new Semaphore(1, true); + private final AtomicLong lstReConTime = new AtomicLong(0); - public NettyClient(Bootstrap bootstrap, String serverIP, - int serverPort, ProxyClientConfig configure) { + public NettyClient(String callerId, + Bootstrap bootstrap, HostInfo hostInfo, ProxyClientConfig configure) { + this.callerId = callerId; this.bootstrap = bootstrap; - this.serverIP = serverIP; - this.serverPort = serverPort; + this.hostInfo = hostInfo; this.configure = configure; - setState(ConnState.INIT); - } - - public Channel getChannel() { - return channel; + setState(CLIENT_STATUS_INIT); } - public void setChannel(Channel channel) { - this.channel = channel; - } - - public boolean connect() { - // Connect to server. - - setState(ConnState.INIT); + public boolean connect(boolean needPrint) { + // Initial status + this.setState(CLIENT_STATUS_INIT); + long curTime = System.currentTimeMillis(); final CountDownLatch awaitLatch = new CountDownLatch(1); - ChannelFuture future = bootstrap.connect(new InetSocketAddress( - serverIP, serverPort)); + // Build connect to server + ChannelFuture future = bootstrap.connect( + new InetSocketAddress(hostInfo.getHostName(), hostInfo.getPortNumber())); future.addListener(new ChannelFutureListener() { public void operationComplete(ChannelFuture arg0) throws Exception { - logger.info("connect ack! {}", serverIP); awaitLatch.countDown(); } }); - try { // Wait until the connection is built. - awaitLatch.await(configure.getConnectTimeoutMs(), - TimeUnit.MILLISECONDS); - } catch (Exception e) { - logger.error("create connect exception! {}", e.getMessage()); - e.printStackTrace(); + awaitLatch.await(configure.getConnectTimeoutMs(), TimeUnit.MILLISECONDS); + } catch (Throwable ex) { + if (conExptCnt.shouldPrint()) { + logger.warn("NettyClient({}) connect to {} exception", + callerId, hostInfo.getReferenceName(), ex); + } return false; } - // Return if no connection is built. if (!future.isSuccess()) { + if (needPrint) { + logger.info("NettyClient({}) connect to {} failure, wast {}ms", + callerId, hostInfo.getReferenceName(), (System.currentTimeMillis() - curTime)); + } return false; } - channel = future.channel(); - setState(ConnState.READY); - logger.info("ip {} stat {}", serverIP, connState); + this.channel = future.channel(); + this.lstSendTime.set(System.currentTimeMillis()); + this.setState(CLIENT_STATUS_READY); + if (needPrint) { + logger.info("NettyClient({}) connect to {} success, wast {}ms", + callerId, channel.toString(), (System.currentTimeMillis() - curTime)); + } return true; } - public boolean close() { - logger.debug("begin to close this channel{}", channel); - final CountDownLatch awaitLatch = new CountDownLatch(1); + public boolean close(boolean needPrint) { boolean ret = true; + String channelStr = ""; + setState(CLIENT_STATUS_DEAD); + long curTime = System.currentTimeMillis(); + final CountDownLatch awaitLatch = new CountDownLatch(1); try { - if (channel != null) { + if (channel == null) { + channelStr = hostInfo.getReferenceName(); + } else { + channelStr = channel.toString(); ChannelFuture future = channel.close(); future.addListener(new ChannelFutureListener() { - public void operationComplete(ChannelFuture arg0) - throws Exception { - logger.info("close client ack {}", serverIP); + public void operationComplete(ChannelFuture arg0) throws Exception { awaitLatch.countDown(); } }); @@ -129,60 +135,127 @@ public void operationComplete(ChannelFuture arg0) ret = false; } } - } catch (Exception e) { - logger.error("close connect {" + serverIP + ":" + serverPort + "} exception! {}", e.getMessage()); - e.printStackTrace(); + } catch (Throwable ex) { + if (conExptCnt.shouldPrint()) { + logger.warn("NettyClient({}) close {} exception", callerId, channelStr, ex); + } ret = false; } finally { - setState(ConnState.DEAD); + this.channel = null; + this.msgInFlight.set(0); + } + if (needPrint) { + if (ret) { + logger.info("NettyClient({}) close {} success, wast {}ms", + this.callerId, channelStr, (System.currentTimeMillis() - curTime)); + } else { + logger.info("NettyClient({}) close {} failure, wast {}ms", + this.callerId, channelStr, (System.currentTimeMillis() - curTime)); + } } - logger.info("end to close {" + serverIP + ":" + serverPort + "} 's channel, bSuccess = " + ret); return ret; } - public void reconnect() { - this.close(); - this.connect(); + public ChannelFuture write(EncodeObject encodeObject) { + ChannelFuture future = null; + try { + future = channel.writeAndFlush(encodeObject); + this.lstSendTime.set(System.currentTimeMillis()); + } catch (Throwable ex) { + if (conExptCnt.shouldPrint()) { + logger.warn("NettyClient({}) write {} exception", callerId, channel.toString(), ex); + } + } + return future; } - public boolean isActive() { - stateLock.lock(); - try { - return (connState == ConnState.READY && channel != null && channel.isOpen() && channel.isActive()); - } catch (Exception e) { - logger.error("channel maybe null!{}", e.getMessage()); + public boolean reconnect(boolean needPrint) { + if (this.isActive() + || this.msgInFlight.get() > 0 + || (System.currentTimeMillis() - this.lstReConTime.get()) < this.configure.getReConnectWaitMs()) { + return false; + } + if (reconSemaphore.tryAcquire()) { + try { + if (this.isActive()) { + return true; + } + this.lstReConTime.set(System.currentTimeMillis()); + this.close(false); + if (this.connect(false)) { + if (needPrint) { + logger.info("NettyClient({}) re-connect to {} success", + callerId, this.channel.toString()); + } + return true; + } else { + if (needPrint) { + logger.info("NettyClient({}) re-connect to {} failure", + callerId, hostInfo.getReferenceName()); + } + return false; + } + } finally { + reconSemaphore.release(); + } + } else { return false; - } finally { - stateLock.unlock(); } - // channel.isOpen(); } - private void setState(ConnState newState) { - stateLock.lock(); - try { - connState = newState; - } catch (Exception e) { - logger.error("setState maybe error!{}", e.getMessage()); - } finally { - stateLock.unlock(); + public String getNodeAddress() { + return hostInfo.getReferenceName(); + } + + public String getServerIP() { + return hostInfo.getHostName(); + } + + public Channel getChannel() { + return channel; + } + + public void setFrozen(ChannelId channelId) { + if (this.channel != null && this.channel.id() == channelId) { + setState(CLIENT_STATUS_FROZEN); } } - private enum ConnState { - INIT, READY, FROZEN, DEAD, BUSY + public void setBusy(ChannelId channelId) { + if (this.channel != null && this.channel.id() == channelId) { + setState(CLIENT_STATUS_BUSY); + } } - public ChannelFuture write(EncodeObject encodeObject) { - // TODO Auto-generated method stub - ChannelFuture future = null; - try { - future = channel.writeAndFlush(encodeObject); - } catch (Exception e) { - logger.error("channel write error {}", e.getMessage()); - e.printStackTrace(); + public boolean isActive() { + return ((this.conStatus.get() == CLIENT_STATUS_READY) + && channel != null && channel.isOpen() && channel.isActive()); + } + + public boolean isIdleClient(long curTime) { + return (curTime - this.lstSendTime.get() >= 30000L); + } + + public boolean tryIncMsgInFlight() { + if (configure.getMaxMsgInFlightPerConn() > 0) { + if (msgInFlight.getAndIncrement() > configure.getMaxMsgInFlightPerConn()) { + msgInFlight.decrementAndGet(); + return false; + } } - return future; + return true; + } + + public void decMsgInFlight() { + if (configure.getMaxMsgInFlightPerConn() > 0) { + if (msgInFlight.decrementAndGet() < 0L) { + logger.warn("NettyClient({}) dec inflight({}) value < 0", callerId, hostInfo.getReferenceName()); + } + } + } + + public long getMsgInFlight() { + return msgInFlight.get(); } @Override @@ -198,29 +271,21 @@ public boolean equals(Object obj) { } NettyClient other = (NettyClient) obj; if (channel == null) { - if (other.channel != null) { - return false; - } - } else if (!channel.equals(other.channel)) { - return false; + return other.channel == null; + } else { + return channel.equals(other.channel); } - return true; - } - - public void setFrozen() { - // TODO Auto-generated method stub - setState(ConnState.FROZEN); - } - public void setBusy() { - setState(ConnState.BUSY); - } - - public double getWeight() { - OperatingSystemMXBean operatingSystemMXBean = - (OperatingSystemMXBean) ManagementFactory.getOperatingSystemMXBean(); - return operatingSystemMXBean.getSystemLoadAverage(); + private void setState(int newState) { + int curState = conStatus.get(); + if (curState == newState) { + return; + } + if (newState == CLIENT_STATUS_DEAD + || (curState == CLIENT_STATUS_INIT && newState == CLIENT_STATUS_READY) + || (curState == CLIENT_STATUS_READY && newState > 0)) { + this.conStatus.compareAndSet(curState, newState); + } } - } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/QueueObject.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/QueueObject.java index 31e53323967..36992131764 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/QueueObject.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/QueueObject.java @@ -20,32 +20,30 @@ import org.apache.inlong.sdk.dataproxy.common.SendMessageCallback; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; public class QueueObject { + private final NettyClient client; + private final AtomicBoolean done = new AtomicBoolean(false); private final long sendTimeInMillis; private final SendMessageCallback callback; private final long timeoutInMillis; private final int size; - public QueueObject(long sendTimeInMillis, - SendMessageCallback callback, - long timeout, - TimeUnit timeUnit) { + public QueueObject(NettyClient client, long sendTimeInMillis, + SendMessageCallback callback, int size, long timeout, TimeUnit timeUnit) { + this.client = client; this.sendTimeInMillis = sendTimeInMillis; this.callback = callback; this.timeoutInMillis = TimeUnit.MILLISECONDS.convert(timeout, timeUnit); - this.size = 1; + this.size = size; } - public QueueObject(long sendTimeInMillis, - SendMessageCallback callback, int size, - long timeout, - TimeUnit timeUnit) { - this.sendTimeInMillis = sendTimeInMillis; - this.callback = callback; - this.timeoutInMillis = TimeUnit.MILLISECONDS.convert(timeout, timeUnit); - this.size = size; + public void done() { + if (client != null && done.compareAndSet(false, true)) { + client.decMsgInFlight(); + } } public long getSendTimeInMillis() { diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/Sender.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/Sender.java index 01ac56a53b9..13920fb7a60 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/Sender.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/Sender.java @@ -24,16 +24,15 @@ import org.apache.inlong.sdk.dataproxy.config.ProxyConfigEntry; import org.apache.inlong.sdk.dataproxy.threads.MetricWorkerThread; import org.apache.inlong.sdk.dataproxy.threads.TimeoutScanThread; +import org.apache.inlong.sdk.dataproxy.utils.LogCounter; +import org.apache.inlong.sdk.dataproxy.utils.Tuple2; import io.netty.channel.Channel; -import org.apache.commons.collections.MapUtils; -import org.apache.commons.lang3.ObjectUtils; +import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.PrintWriter; -import java.io.StringWriter; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -42,25 +41,30 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.atomic.AtomicLong; public class Sender { - private static final Logger LOGGER = LoggerFactory.getLogger(Sender.class); + private static final Logger logger = LoggerFactory.getLogger(Sender.class); + private static final LogCounter exptCnt = new LogCounter(10, 100000, 60 * 1000L); + private static final LogCounter unwritableExptCnt = new LogCounter(10, 100000, 60 * 1000L); + private static final LogCounter reqChkLoggCount = new LogCounter(10, 100000, 60 * 1000L); + private static final AtomicLong senderIdGen = new AtomicLong(0L); /* Store the callback used by asynchronously message sending. */ private final ConcurrentHashMap> callbacks = new ConcurrentHashMap<>(); /* Store the synchronous message sending invocations. */ private final ConcurrentHashMap syncCallables = new ConcurrentHashMap<>(); - private final ConcurrentHashMap chooseProxy = new ConcurrentHashMap<>(); - private final ReentrantLock stateLock = new ReentrantLock(); private final ExecutorService threadPool; private final int asyncCallbackMaxSize; private final AtomicInteger currentBufferSize = new AtomicInteger(0); private final TimeoutScanThread scanThread; + private final AtomicBoolean started = new AtomicBoolean(false); private final ClientMgr clientMgr; + private final String instanceId; private final ProxyClientConfig configure; private MetricWorkerThread metricWorker = null; private int clusterId = -1; @@ -74,58 +78,52 @@ public Sender(ProxyClientConfig configure) throws Exception { */ public Sender(ProxyClientConfig configure, ThreadFactory selfDefineFactory) throws Exception { this.configure = configure; + this.instanceId = "sender-" + senderIdGen.incrementAndGet(); this.asyncCallbackMaxSize = configure.getTotalAsyncCallbackSize(); this.threadPool = Executors.newCachedThreadPool(); this.clientMgr = new ClientMgr(configure, this, selfDefineFactory); - ProxyConfigEntry proxyConfigEntry = null; + ProxyConfigEntry proxyConfigEntry; try { - proxyConfigEntry = this.clientMgr.getGroupIdConfigureInfo(); + proxyConfigEntry = this.clientMgr.getGroupIdConfigure(); setClusterId(proxyConfigEntry.getClusterId()); } catch (Throwable e) { - if (configure.isReadProxyIPFromLocal()) { + if (configure.isOnlyUseLocalProxyConfig()) { throw new Exception("Get local proxy configure failure!", e.getCause()); } else { throw new Exception("Visit manager error!", e.getCause()); } } if (!proxyConfigEntry.isInterVisit()) { - if (!configure.isNeedAuthentication()) { + if (!configure.isEnableAuthentication()) { throw new Exception("In OutNetwork isNeedAuthentication must be true!"); } if (!configure.isNeedDataEncry()) { throw new Exception("In OutNetwork isNeedDataEncry must be true!"); } } - scanThread = new TimeoutScanThread(callbacks, currentBufferSize, configure, clientMgr); - scanThread.start(); - + scanThread = new TimeoutScanThread(this, configure); if (configure.isEnableMetric()) { metricWorker = new MetricWorkerThread(configure, this); - metricWorker.start(); } - - LOGGER.info("proxy sdk is starting!"); + logger.info("Sender({}) instance initialized!", this.instanceId); } - private void checkCallbackList() { - // max wait for 1 min - LOGGER.info("checking call back list before close, current size is {}", - currentBufferSize.get()); - int count = 0; - try { - while (currentBufferSize.get() > 0 && count < 60) { - TimeUnit.SECONDS.sleep(1); - count += 1; - } - if (currentBufferSize.get() > 0) { - LOGGER.warn("callback not empty {}, please check it", currentBufferSize.get()); - } - } catch (Exception ex) { - LOGGER.error("exception while checking callback list", ex); + public void start() { + if (!started.compareAndSet(false, true)) { + return; } + this.clientMgr.start(); + this.scanThread.start(); + if (this.configure.isEnableMetric()) { + this.metricWorker.start(); + } + logger.info("Sender({}) instance started!", this.instanceId); } public void close() { + if (!started.compareAndSet(true, false)) { + return; + } checkCallbackList(); scanThread.shutDown(); clientMgr.shutDown(); @@ -133,31 +131,11 @@ public void close() { if (configure.isEnableMetric()) { metricWorker.close(); } - } - - public String getExceptionStack(Throwable e) { - StringWriter sw = new StringWriter(); - PrintWriter pw = new PrintWriter(sw); - String exceptStr = null; - try { - e.printStackTrace(pw); - exceptStr = sw.toString(); - } catch (Exception ex) { - LOGGER.error(getExceptionStack(ex)); - } finally { - try { - pw.close(); - sw.close(); - } catch (Exception ex) { - LOGGER.error(getExceptionStack(ex)); - } - } - return exceptStr; + logger.info("Sender({}) instance stopped!", this.instanceId); } /* Used for asynchronously message sending. */ public void notifyCallback(Channel channel, String messageId, SendResult result) { - LOGGER.debug("Channel = {} , ack messageId = {}", channel, messageId); if (channel == null) { return; } @@ -171,46 +149,7 @@ public void notifyCallback(Channel channel, String messageId, SendResult result) } callback.getCallback().onMessageAck(result); currentBufferSize.decrementAndGet(); - } - - private SendResult syncSendInternalMessage(NettyClient client, - EncodeObject encodeObject, String msgUUID) - throws ExecutionException, InterruptedException, TimeoutException { - if (client == null) { - return SendResult.NO_CONNECTION; - } - if (isNotValidateAttr(encodeObject.getCommonattr(), encodeObject.getAttributes())) { - LOGGER.error("error attr format {} {}", encodeObject.getCommonattr(), - encodeObject.getAttributes()); - return SendResult.INVALID_ATTRIBUTES; - } - if (encodeObject.getMsgtype() == 7) { - int groupIdnum = 0; - int streamIdnum = 0; - if (encodeObject.getGroupId().equals(clientMgr.getGroupId())) { - groupIdnum = clientMgr.getGroupIdNum(); - streamIdnum = clientMgr.getStreamIdMap().get(encodeObject.getStreamId()) != null - ? clientMgr.getStreamIdMap().get(encodeObject.getStreamId()) - : 0; - } - encodeObject.setGroupIdNum(groupIdnum); - encodeObject.setStreamIdNum(streamIdnum); - if (groupIdnum == 0 || streamIdnum == 0) { - encodeObject.setGroupIdTransfer(false); - } - } - if (this.configure.isNeedDataEncry()) { - encodeObject.setEncryptEntry(true, configure.getUserName(), clientMgr.getEncryptConfigEntry()); - } else { - encodeObject.setEncryptEntry(false, null, null); - } - encodeObject.setMsgUUID(msgUUID); - SyncMessageCallable callable = new SyncMessageCallable(client, encodeObject, - configure.getRequestTimeoutMs(), TimeUnit.MILLISECONDS); - syncCallables.put(encodeObject.getMessageId(), callable); - - Future future = threadPool.submit(callable); - return future.get(configure.getRequestTimeoutMs(), TimeUnit.MILLISECONDS); + callback.done(); } /** @@ -222,53 +161,99 @@ private SendResult syncSendInternalMessage(NettyClient client, * @return */ public SendResult syncSendMessage(EncodeObject encodeObject, String msgUUID) { + if (!started.get()) { + return SendResult.SENDER_CLOSED; + } if (configure.isEnableMetric()) { metricWorker.recordNumByKey(encodeObject.getMessageId(), encodeObject.getGroupId(), - encodeObject.getStreamId(), Utils.getLocalIp(), encodeObject.getDt(), + encodeObject.getStreamId(), IpUtils.getLocalIp(), encodeObject.getDt(), encodeObject.getPackageTime(), encodeObject.getRealCnt()); } - NettyClient client = clientMgr.getClient(clientMgr.getLoadBalance(), encodeObject); - SendResult message = null; + SendResult message; + Tuple2 clientResult = null; try { - message = syncSendInternalMessage(client, encodeObject, msgUUID); + MutableBoolean allClientMaxInFlight = new MutableBoolean(false); + clientResult = clientMgr.getClientByRoundRobin(allClientMaxInFlight); + if (allClientMaxInFlight.booleanValue()) { + return SendResult.MAX_FLIGHT_ON_ALL_CONNECTION; + } + if (clientResult.getF0() != SendResult.OK) { + return clientResult.getF0(); + } + if (!clientResult.getF1().getChannel().isWritable()) { + if (unwritableExptCnt.shouldPrint()) { + logger.warn("Sender({}) channel={} touch write_over_water", + getInstanceId(), clientResult.getF1().getChannel()); + } + return SendResult.WRITE_OVER_WATERMARK; + } + if (isNotValidateAttr(encodeObject.getCommonattr(), encodeObject.getAttributes())) { + if (reqChkLoggCount.shouldPrint()) { + logger.warn("Sender({}) found error attr format {} {}", + getInstanceId(), encodeObject.getCommonattr(), encodeObject.getAttributes()); + } + return SendResult.INVALID_ATTRIBUTES; + } + if (encodeObject.getMsgtype() == 7) { + if (clientMgr.isIdTransNum() + && encodeObject.getGroupId().equals(clientMgr.getGroupId())) { + encodeObject.setGroupIdAndStreamIdNum(clientMgr.getGroupIdNum(), + clientMgr.getStreamIdNum(encodeObject.getStreamId())); + } + } + if (this.configure.isNeedDataEncry()) { + encodeObject.setEncryptEntry(true, + configure.getAuthSecretId(), clientMgr.getEncryptConfigureInfo()); + } else { + encodeObject.setEncryptEntry(false, null, null); + } + encodeObject.setMsgUUID(msgUUID); + SyncMessageCallable callable = new SyncMessageCallable(clientResult.getF1(), + encodeObject, configure.getRequestTimeoutMs(), TimeUnit.MILLISECONDS); + syncCallables.put(encodeObject.getMessageId(), callable); + Future future = threadPool.submit(callable); + message = future.get(configure.getRequestTimeoutMs(), TimeUnit.MILLISECONDS); } catch (InterruptedException e) { - // TODO Auto-generated catch block - LOGGER.error("send message error {} ", getExceptionStack(e)); syncCallables.remove(encodeObject.getMessageId()); return SendResult.THREAD_INTERRUPT; } catch (ExecutionException e) { - // TODO Auto-generated catch block - LOGGER.error("ExecutionException {} ", getExceptionStack(e)); syncCallables.remove(encodeObject.getMessageId()); + if (exptCnt.shouldPrint()) { + logger.warn("Sender({}) sync send msg throw ExecutionException", + getInstanceId(), e); + } return SendResult.UNKOWN_ERROR; } catch (TimeoutException e) { - // TODO Auto-generated catch block - LOGGER.error("TimeoutException {} ", getExceptionStack(e)); - // e.printStackTrace(); SyncMessageCallable syncMessageCallable = syncCallables.remove(encodeObject.getMessageId()); if (syncMessageCallable != null) { NettyClient tmpClient = syncMessageCallable.getClient(); if (tmpClient != null) { Channel curChannel = tmpClient.getChannel(); if (curChannel != null) { - LOGGER.error("channel maybe busy {}", curChannel); scanThread.addTimeoutChannel(curChannel); } } } + if (exptCnt.shouldPrint()) { + logger.warn("Sender({}) sync send msg throw TimeoutException", getInstanceId(), e); + } return SendResult.TIMEOUT; } catch (Throwable e) { - LOGGER.error("syncSendMessage exception {} ", getExceptionStack(e)); syncCallables.remove(encodeObject.getMessageId()); + if (exptCnt.shouldPrint()) { + logger.warn("Sender({}) sync send msg throw exception", getInstanceId(), e); + } return SendResult.UNKOWN_ERROR; + } finally { + if (clientResult != null && clientResult.getF1() != null) { + clientResult.getF1().decMsgInFlight(); + } } if (message == null) { syncCallables.remove(encodeObject.getMessageId()); return SendResult.UNKOWN_ERROR; } - if (client != null) { - scanThread.resetTimeoutChannel(client.getChannel()); - } + scanThread.resetTimeoutChannel(clientResult.getF1().getChannel()); if (message == SendResult.OK) { if (configure.isEnableMetric()) { metricWorker.recordSuccessByMessageId(encodeObject.getMessageId()); @@ -326,63 +311,115 @@ private boolean validAttribute(String attr) { */ public void asyncSendMessage(EncodeObject encodeObject, SendMessageCallback callback, String msgUUID, long timeout, TimeUnit timeUnit) throws ProxysdkException { + if (!started.get()) { + if (callback != null) { + callback.onMessageAck(SendResult.SENDER_CLOSED); + return; + } else { + throw new ProxysdkException(SendResult.SENDER_CLOSED.toString()); + } + } if (configure.isEnableMetric()) { metricWorker.recordNumByKey(encodeObject.getMessageId(), encodeObject.getGroupId(), - encodeObject.getStreamId(), Utils.getLocalIp(), encodeObject.getPackageTime(), + encodeObject.getStreamId(), IpUtils.getLocalIp(), encodeObject.getPackageTime(), encodeObject.getDt(), encodeObject.getRealCnt()); } // send message package time - NettyClient client = clientMgr.getClient(clientMgr.getLoadBalance(), encodeObject); - if (client == null) { - throw new ProxysdkException(SendResult.NO_CONNECTION.toString()); + MutableBoolean allClientMaxInFlight = new MutableBoolean(false); + Tuple2 clientResult = + clientMgr.getClientByRoundRobin(allClientMaxInFlight); + if (allClientMaxInFlight.booleanValue()) { + if (callback != null) { + callback.onMessageAck(SendResult.MAX_FLIGHT_ON_ALL_CONNECTION); + return; + } else { + throw new ProxysdkException(SendResult.MAX_FLIGHT_ON_ALL_CONNECTION.toString()); + } + } + if (clientResult.getF0() != SendResult.OK) { + if (callback != null) { + callback.onMessageAck(clientResult.getF0()); + return; + } else { + throw new ProxysdkException(clientResult.getF0().toString()); + } + } + if (!clientResult.getF1().getChannel().isWritable()) { + if (unwritableExptCnt.shouldPrint()) { + logger.warn("Sender({}) found channel={} touch write_over_water", + getInstanceId(), clientResult.getF1().getChannel()); + } + clientResult.getF1().decMsgInFlight(); + if (callback != null) { + callback.onMessageAck(SendResult.WRITE_OVER_WATERMARK); + return; + } else { + throw new ProxysdkException(SendResult.WRITE_OVER_WATERMARK.toString()); + } } if (currentBufferSize.get() >= asyncCallbackMaxSize) { - throw new ProxysdkException("ASYNC_CALLBACK_BUFFER_FULL"); + clientResult.getF1().decMsgInFlight(); + if (callback != null) { + callback.onMessageAck(SendResult.ASYNC_CALLBACK_BUFFER_FULL); + return; + } else { + throw new ProxysdkException(SendResult.ASYNC_CALLBACK_BUFFER_FULL.toString()); + } } if (isNotValidateAttr(encodeObject.getCommonattr(), encodeObject.getAttributes())) { - LOGGER.error("error attr format {} {}", encodeObject.getCommonattr(), - encodeObject.getAttributes()); - throw new ProxysdkException(SendResult.INVALID_ATTRIBUTES.toString()); + if (reqChkLoggCount.shouldPrint()) { + logger.warn("Sender({}) found error attr format {} {}", + getInstanceId(), encodeObject.getCommonattr(), encodeObject.getAttributes()); + } + clientResult.getF1().decMsgInFlight(); + if (callback != null) { + callback.onMessageAck(SendResult.INVALID_ATTRIBUTES); + return; + } else { + throw new ProxysdkException(SendResult.INVALID_ATTRIBUTES.toString()); + } } int size = 1; if (currentBufferSize.incrementAndGet() >= asyncCallbackMaxSize) { currentBufferSize.decrementAndGet(); - throw new ProxysdkException("ASYNC_CALLBACK_BUFFER_FULL"); + clientResult.getF1().decMsgInFlight(); + if (callback != null) { + callback.onMessageAck(SendResult.ASYNC_CALLBACK_BUFFER_FULL); + return; + } else { + throw new ProxysdkException(SendResult.ASYNC_CALLBACK_BUFFER_FULL.toString()); + } } ConcurrentHashMap msgQueueMap = - callbacks.computeIfAbsent(client.getChannel(), (k) -> new ConcurrentHashMap<>()); + callbacks.computeIfAbsent(clientResult.getF1().getChannel(), (k) -> new ConcurrentHashMap<>()); QueueObject queueObject = msgQueueMap.putIfAbsent(encodeObject.getMessageId(), - new QueueObject(System.currentTimeMillis(), callback, size, timeout, timeUnit)); + new QueueObject(clientResult.getF1(), System.currentTimeMillis(), callback, size, timeout, timeUnit)); if (queueObject != null) { - LOGGER.warn("message id {} has existed.", encodeObject.getMessageId()); + if (reqChkLoggCount.shouldPrint()) { + logger.warn("Sender({}) found message id {} has existed.", + getInstanceId(), encodeObject.getMessageId()); + } } if (encodeObject.getMsgtype() == 7) { - int groupIdnum = 0; - int streamIdnum = 0; - if ((clientMgr.getGroupId().length() != 0) && (encodeObject.getGroupId().equals(clientMgr.getGroupId()))) { - groupIdnum = clientMgr.getGroupIdNum(); - streamIdnum = (clientMgr.getStreamIdMap().get(encodeObject.getStreamId()) != null) ? clientMgr - .getStreamIdMap().get(encodeObject.getStreamId()) : 0; - } - encodeObject.setGroupIdNum(groupIdnum); - encodeObject.setStreamIdNum(streamIdnum); - if (groupIdnum == 0 || streamIdnum == 0) { - encodeObject.setGroupIdTransfer(false); + if (clientMgr.isIdTransNum() + && encodeObject.getGroupId().equals(clientMgr.getGroupId())) { + encodeObject.setGroupIdAndStreamIdNum(clientMgr.getGroupIdNum(), + clientMgr.getStreamIdNum(encodeObject.getStreamId())); } } if (this.configure.isNeedDataEncry()) { - encodeObject.setEncryptEntry(true, configure.getUserName(), clientMgr.getEncryptConfigEntry()); + encodeObject.setEncryptEntry(true, + configure.getAuthSecretId(), clientMgr.getEncryptConfigureInfo()); } else { encodeObject.setEncryptEntry(false, null, null); } encodeObject.setMsgUUID(msgUUID); - client.write(encodeObject); + clientResult.getF1().write(encodeObject); } /* Deal with feedback. */ public void notifyFeedback(Channel channel, EncodeObject response) { String messageId = response.getMessageId(); - chooseProxy.remove(messageId); SyncMessageCallable callable = syncCallables.remove(messageId); SendResult result = response.getSendResult(); if (result == SendResult.OK) { @@ -390,7 +427,10 @@ public void notifyFeedback(Channel channel, EncodeObject response) { metricWorker.recordSuccessByMessageId(messageId); } } else { - LOGGER.error("{} exception happens, error message {}", channel, response.getErrMsg()); + if (logger.isDebugEnabled()) { + logger.debug("Sender({}) send message to {} exception, errMsg={}", + getInstanceId(), channel, response.getErrMsg()); + } } if (callable != null) { // for syncSend callable.update(result); @@ -405,7 +445,10 @@ public void notifyConnectionDisconnected(Channel channel) { if (channel == null) { return; } - LOGGER.info("channel {} connection is disconnected!", channel); + if (logger.isDebugEnabled()) { + logger.debug("Sender({}) found channel {} connection is disconnected!", + getInstanceId(), channel); + } try { ConcurrentHashMap msgQueueMap = callbacks.remove(channel); if (msgQueueMap != null) { @@ -416,11 +459,15 @@ public void notifyConnectionDisconnected(Channel channel) { } queueObject.getCallback().onMessageAck(SendResult.CONNECTION_BREAK); currentBufferSize.decrementAndGet(); + queueObject.done(); } msgQueueMap.clear(); } } catch (Throwable e2) { - LOGGER.info("process channel {} disconnected callbacks throw error,", channel, e2); + if (exptCnt.shouldPrint()) { + logger.warn("Sender({}) process channel disconnected {} throw error", + getInstanceId(), channel, e2); + } } try { @@ -447,7 +494,10 @@ public void notifyConnectionDisconnected(Channel channel) { } } } catch (Throwable e) { - LOGGER.info("process channel {} disconnected syncCallables throw error,", channel, e); + if (exptCnt.shouldPrint()) { + logger.warn("Sender({}) process channel {} disconnected syncCallables throw error", + getInstanceId(), channel, e); + } } } @@ -456,28 +506,27 @@ public void waitForAckForChannel(Channel channel) { if (channel == null) { return; } - LOGGER.info("wait for ack for channel {}", channel); + long startTime = System.currentTimeMillis(); + ConcurrentHashMap queueObjMap = callbacks.get(channel); + if (queueObjMap == null || queueObjMap.isEmpty()) { + return; + } try { - ConcurrentHashMap queueObjMap = callbacks.get(channel); - if (queueObjMap != null) { - while (true) { - if (queueObjMap.isEmpty()) { - LOGGER.info("this channel {} is empty!", channel); - break; - } - try { - Thread.sleep(1000); - } catch (InterruptedException e) { - // TODO Auto-generated catch block - LOGGER.error("wait for ack for channel {}, error {}", - channel, e.getMessage()); - e.printStackTrace(); - } + while (!queueObjMap.isEmpty()) { + try { + Thread.sleep(100); + } catch (InterruptedException ex1) { + // + } + if (System.currentTimeMillis() - startTime >= configure.getConCloseWaitPeriodMs()) { + break; } } - LOGGER.info("waitForAckForChannel finished , channel is {}", channel); - } catch (Throwable e) { - LOGGER.error("waitForAckForChannel exception, channel is {}", channel, e); + } catch (Throwable ex) { + if (exptCnt.shouldPrint()) { + logger.warn("Sender({}) waitForAckForChannel channel {} throw error", + getInstanceId(), channel, ex); + } } } @@ -486,10 +535,6 @@ public void clearCallBack() { callbacks.clear(); } - public void clearCallBackByChannel(Channel channel) { - callbacks.remove(channel); - } - public int getClusterId() { return clusterId; } @@ -498,26 +543,36 @@ public void setClusterId(int clusterId) { this.clusterId = clusterId; } - /** - * check whether clientChannel is idle; if idle, need send hb to keep alive - * - * @param client - * @return - */ - public boolean isIdleClient(NettyClient client) { - Channel channel = client.getChannel(); - // used by async send - if (callbacks.contains(channel) && MapUtils.isNotEmpty(callbacks.get(channel))) { - return false; - } - // used by sync send - for (SyncMessageCallable syncCallBack : syncCallables.values()) { - if (ObjectUtils.equals(client, syncCallBack.getClient())) { - return false; - } - } + public String getInstanceId() { + return instanceId; + } + + public AtomicInteger getCurrentBufferSize() { + return currentBufferSize; + } - return true; + public ConcurrentHashMap> getCallbacks() { + return callbacks; } + public ClientMgr getClientMgr() { + return clientMgr; + } + + private void checkCallbackList() { + // max wait for 1 min + try { + long startTime = System.currentTimeMillis(); + while (currentBufferSize.get() > 0 + && System.currentTimeMillis() - startTime < configure.getConCloseWaitPeriodMs()) { + TimeUnit.MILLISECONDS.sleep(300); + } + if (currentBufferSize.get() > 0) { + logger.warn("Sender({}) callback size({}) not empty, force quit!", + getInstanceId(), currentBufferSize.get()); + } + } catch (Throwable ex) { + // + } + } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/SequentialID.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/SequentialID.java index a577f0038c8..0ad2813448c 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/SequentialID.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/SequentialID.java @@ -24,11 +24,10 @@ public class SequentialID { private static final SecureRandom sRandom = new SecureRandom( Long.toString(System.nanoTime()).getBytes()); - private final String ip; + private final String ip = IpUtils.getLocalIp(); private final AtomicInteger id = new AtomicInteger(sRandom.nextInt()); - public SequentialID(String theIp) { - ip = theIp; + public SequentialID() { } public String getNextId() { diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/SyncMessageCallable.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/SyncMessageCallable.java index 8e00ff8c3ed..bce2ad44683 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/SyncMessageCallable.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/SyncMessageCallable.java @@ -19,6 +19,7 @@ import org.apache.inlong.sdk.dataproxy.codec.EncodeObject; import org.apache.inlong.sdk.dataproxy.common.SendResult; +import org.apache.inlong.sdk.dataproxy.utils.LogCounter; import io.netty.channel.ChannelFuture; import org.slf4j.Logger; @@ -30,8 +31,8 @@ public class SyncMessageCallable implements Callable { - private static final Logger logger = LoggerFactory - .getLogger(SyncMessageCallable.class); + private static final Logger logger = LoggerFactory.getLogger(SyncMessageCallable.class); + private static final LogCounter exptCnt = new LogCounter(10, 100000, 60 * 1000L); private final NettyClient client; private final CountDownLatch awaitLatch = new CountDownLatch(1); @@ -55,13 +56,16 @@ public void update(SendResult message) { } public SendResult call() throws Exception { - // TODO Auto-generated method stub try { + if (!client.getChannel().isWritable()) { + return SendResult.WRITE_OVER_WATERMARK; + } ChannelFuture channelFuture = client.write(encodeObject); awaitLatch.await(timeout, timeUnit); - } catch (Exception e) { - logger.error("SendResult call", e); - e.printStackTrace(); + } catch (Throwable ex) { + if (exptCnt.shouldPrint()) { + logger.warn("SyncMessageCallable write data throw exception", ex); + } return SendResult.UNKOWN_ERROR; } return message; diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/TimeScanObject.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/TimeScanObject.java index 01d36c23fbb..19291e4336d 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/TimeScanObject.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/network/TimeScanObject.java @@ -22,8 +22,8 @@ public class TimeScanObject { - private AtomicInteger count = new AtomicInteger(0); - private AtomicLong time = new AtomicLong(0); + private final AtomicInteger count = new AtomicInteger(0); + private final AtomicLong time = new AtomicLong(0); public TimeScanObject() { this.count.set(0); diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/pb/context/SinkContext.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/pb/context/SinkContext.java index 4c2cd285e8b..366bf5ef541 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/pb/context/SinkContext.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/pb/context/SinkContext.java @@ -20,7 +20,7 @@ import org.apache.inlong.common.metric.MetricRegister; import org.apache.inlong.sdk.dataproxy.pb.metrics.SdkMetricItem; import org.apache.inlong.sdk.dataproxy.pb.metrics.SdkMetricItemSet; -import org.apache.inlong.sdk.dataproxy.utils.IpUtils; +import org.apache.inlong.sdk.dataproxy.pb.network.IpUtils; import org.apache.flume.Channel; import org.apache.flume.Context; diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/IpUtils.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/pb/network/IpUtils.java similarity index 99% rename from inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/IpUtils.java rename to inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/pb/network/IpUtils.java index 930f2ecc985..cea17bf9d54 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/IpUtils.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/pb/network/IpUtils.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.inlong.sdk.dataproxy.utils; +package org.apache.inlong.sdk.dataproxy.pb.network; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/ManagerFetcherThread.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/ManagerFetcherThread.java deleted file mode 100644 index 1424c3cb8f4..00000000000 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/ManagerFetcherThread.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.inlong.sdk.dataproxy.threads; - -import org.apache.inlong.sdk.dataproxy.ProxyClientConfig; -import org.apache.inlong.sdk.dataproxy.utils.ServiceDiscoveryUtils; - -import org.apache.commons.lang3.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.concurrent.TimeUnit; - -/** - * manager fetch thread - */ -public class ManagerFetcherThread extends Thread { - - private final Logger logger = LoggerFactory.getLogger(ManagerFetcherThread.class); - private volatile boolean isShutdown; - private final ProxyClientConfig proxyClientConfig; - - public ManagerFetcherThread(ProxyClientConfig proxyClientConfig) { - isShutdown = false; - this.proxyClientConfig = proxyClientConfig; - this.setDaemon(true); - this.setName("ManagerFetcherThread"); - } - - public void shutdown() { - logger.info("Begin to shutdown ManagerFetcherThread."); - isShutdown = true; - } - - @Override - public void run() { - logger.info("ManagerFetcherThread Thread=" + Thread.currentThread().getId() + " started !"); - while (!isShutdown) { - try { - String managerIpList = ServiceDiscoveryUtils.getManagerIpList(proxyClientConfig); - if (StringUtils.isBlank(managerIpList)) { - logger.error("ManagerFetcher get managerIpList is blank."); - } else { - ServiceDiscoveryUtils.updateManagerInfo2Local(managerIpList, - proxyClientConfig.getManagerIpLocalPath()); - } - TimeUnit.MILLISECONDS.sleep((long) proxyClientConfig.getProxyUpdateIntervalMinutes() * 60 * 1000); - } catch (Throwable e) { - logger.error("ManagerFetcher get or save managerIpList occur error,", e); - } - } - } -} \ No newline at end of file diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/MetricWorkerThread.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/MetricWorkerThread.java index 8d79119dece..39f8562b52e 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/MetricWorkerThread.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/MetricWorkerThread.java @@ -24,9 +24,9 @@ import org.apache.inlong.sdk.dataproxy.metric.MessageRecord; import org.apache.inlong.sdk.dataproxy.metric.MetricConfig; import org.apache.inlong.sdk.dataproxy.metric.MetricTimeNumSummary; +import org.apache.inlong.sdk.dataproxy.network.IpUtils; import org.apache.inlong.sdk.dataproxy.network.Sender; import org.apache.inlong.sdk.dataproxy.network.SequentialID; -import org.apache.inlong.sdk.dataproxy.network.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,7 +47,7 @@ public class MetricWorkerThread extends Thread implements Closeable { private static final String DEFAULT_KEY_SPLITTER = "#"; private final Logger logger = LoggerFactory.getLogger(MetricWorkerThread.class); - private final SequentialID idGenerator = new SequentialID(Utils.getLocalIp()); + private final SequentialID idGenerator = new SequentialID(); private final ConcurrentHashMap metricValueCache = new ConcurrentHashMap<>(); private final ConcurrentHashMap metricPackTimeMap = new ConcurrentHashMap<>(); private final ConcurrentHashMap metricDtMap = new ConcurrentHashMap<>(); @@ -208,7 +208,7 @@ private void sendSingleLine(String line, String streamId, long dtTime) { EncodeObject encodeObject = new EncodeObject(Collections.singletonList(line.getBytes()), 7, false, false, false, dtTime, idGenerator.getNextInt(), - metricConfig.getMetricGroupId(), streamId, "", "", Utils.getLocalIp()); + metricConfig.getMetricGroupId(), streamId, "", "", IpUtils.getLocalIp()); MetricSendCallBack callBack = new MetricSendCallBack(encodeObject); tryToSendMetricToManager(encodeObject, callBack); } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/TimeoutScanThread.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/TimeoutScanThread.java index f9e49802643..f53af9b7aa8 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/TimeoutScanThread.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/threads/TimeoutScanThread.java @@ -19,17 +19,18 @@ import org.apache.inlong.sdk.dataproxy.ProxyClientConfig; import org.apache.inlong.sdk.dataproxy.common.SendResult; -import org.apache.inlong.sdk.dataproxy.network.ClientMgr; import org.apache.inlong.sdk.dataproxy.network.QueueObject; +import org.apache.inlong.sdk.dataproxy.network.Sender; import org.apache.inlong.sdk.dataproxy.network.TimeScanObject; +import org.apache.inlong.sdk.dataproxy.utils.LogCounter; import io.netty.channel.Channel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; /** * Daemon threads to check timeout for asynchronous callback. @@ -37,30 +38,27 @@ public class TimeoutScanThread extends Thread { private static final int MAX_CHANNEL_TIMEOUT = 5 * 60 * 1000; - private final Logger logger = LoggerFactory.getLogger(TimeoutScanThread.class); - private final ConcurrentHashMap> callbacks; - private final AtomicInteger currentBufferSize; - private final ProxyClientConfig config; - private final ClientMgr clientMgr; - private final ConcurrentHashMap timeoutChannelStat = new ConcurrentHashMap<>(); + private static final Logger logger = LoggerFactory.getLogger(TimeoutScanThread.class); + private static final LogCounter exptCnt = new LogCounter(10, 100000, 60 * 1000L); private volatile boolean bShutDown = false; private long printCount = 0; + private final ProxyClientConfig config; + private final Sender sender; + private final ConcurrentHashMap timeoutChannelStat = new ConcurrentHashMap<>(); - public TimeoutScanThread(ConcurrentHashMap> callbacks, - AtomicInteger currentBufferSize, ProxyClientConfig config, ClientMgr clientMgr) { - bShutDown = false; - printCount = 0; - this.callbacks = callbacks; - this.currentBufferSize = currentBufferSize; + public TimeoutScanThread(Sender sender, ProxyClientConfig config) { + this.bShutDown = false; this.config = config; - this.clientMgr = clientMgr; + this.sender = sender; this.setDaemon(true); - this.setName("TimeoutScanThread"); + this.setName("TimeoutScanThread-" + this.sender.getInstanceId()); + logger.info("TimeoutScanThread({}) started", this.sender.getInstanceId()); } public void shutDown() { - logger.info("begin to shut down TimeoutScanThread!"); - bShutDown = true; + this.bShutDown = true; + this.interrupt(); + logger.info("TimeoutScanThread({}) shutdown!", this.sender.getInstanceId()); } /** @@ -69,17 +67,18 @@ public void shutDown() { * @param channel */ public void addTimeoutChannel(Channel channel) { - if (channel != null) { - TimeScanObject timeScanObject = timeoutChannelStat.get(channel); + if (channel == null) { + return; + } + TimeScanObject timeScanObject = timeoutChannelStat.get(channel); + if (timeScanObject == null) { + TimeScanObject tmpTimeObj = new TimeScanObject(); + timeScanObject = timeoutChannelStat.putIfAbsent(channel, tmpTimeObj); if (timeScanObject == null) { - TimeScanObject tmpTimeObj = new TimeScanObject(); - timeScanObject = timeoutChannelStat.putIfAbsent(channel, tmpTimeObj); - if (timeScanObject == null) { - timeScanObject = tmpTimeObj; - } + timeScanObject = tmpTimeObj; } - timeScanObject.incrementAndGet(); } + timeScanObject.incrementAndGet(); } /** @@ -88,11 +87,12 @@ public void addTimeoutChannel(Channel channel) { * @param channel */ public void resetTimeoutChannel(Channel channel) { - if (channel != null) { - TimeScanObject timeScanObject = timeoutChannelStat.get(channel); - if (timeScanObject != null) { - timeScanObject.updateCountToZero(); - } + if (channel == null) { + return; + } + TimeScanObject timeScanObject = timeoutChannelStat.get(channel); + if (timeScanObject != null) { + timeScanObject.updateCountToZero(); } } @@ -112,12 +112,10 @@ private void checkTimeoutChannel() { if (System.currentTimeMillis() - timeScanObject.getTime() > MAX_CHANNEL_TIMEOUT) { timeoutChannelStat.remove(tmpChannel); } else { - if (timeScanObject.getCurTimeoutCount() > config.getMaxTimeoutCnt()) { timeoutChannelStat.remove(tmpChannel); if (tmpChannel.isOpen() && tmpChannel.isActive()) { - clientMgr.setConnectionBusy(tmpChannel); - logger.error("this client {} is busy!", tmpChannel); + sender.getClientMgr().setConnectionBusy(tmpChannel); } } } @@ -143,7 +141,8 @@ private void checkMessageIdBasedCallbacks(Channel channel, QueueObject queueObject1 = messageIdCallbacks.remove(messageId); if (queueObject1 != null) { queueObject1.getCallback().onMessageAck(SendResult.TIMEOUT); - currentBufferSize.decrementAndGet(); + sender.getCurrentBufferSize().decrementAndGet(); + queueObject.done(); } addTimeoutChannel(channel); } @@ -152,31 +151,28 @@ private void checkMessageIdBasedCallbacks(Channel channel, @Override public void run() { - logger.info("TimeoutScanThread Thread=" + Thread.currentThread().getId() + " started !"); + logger.info("TimeoutScanThread({}) thread started!", sender.getInstanceId()); while (!bShutDown) { try { - for (Channel channel : callbacks.keySet()) { - ConcurrentHashMap msgQueueMap = - channel != null ? callbacks.get(channel) : null; - if (msgQueueMap == null) { + for (Map.Entry> entry : sender.getCallbacks() + .entrySet()) { + if (entry == null || entry.getKey() == null || entry.getValue() == null) { continue; } - checkMessageIdBasedCallbacks(channel, msgQueueMap); + checkMessageIdBasedCallbacks(entry.getKey(), entry.getValue()); } checkTimeoutChannel(); TimeUnit.SECONDS.sleep(1); - } catch (Throwable e) { - if (!bShutDown) { - logger.error("TimeoutScanThread exception {}", e.getMessage()); - } else { - logger.warn("TimeoutScanThread exception {}", e.getMessage()); + } catch (Throwable ex) { + if (exptCnt.shouldPrint()) { + logger.warn("TimeoutScanThread({}) throw exception", sender.getInstanceId(), ex); } } - if (printCount++ % 20 == 0) { - logger.info("TimeoutScanThread thread=" + Thread.currentThread().getId() - + "'s currentBufferSize = " + currentBufferSize.get()); + if (printCount++ % 60 == 0) { + logger.info("TimeoutScanThread({}) scan, currentBufferSize={}", + sender.getInstanceId(), sender.getCurrentBufferSize().get()); } } - logger.info("TimeoutScanThread Thread=" + Thread.currentThread().getId() + " existed !"); + logger.info("TimeoutScanThread({}) thread existed !", sender.getInstanceId()); } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/EventLoopUtil.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/EventLoopUtil.java index bf26d3fc598..eb99c43b9eb 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/EventLoopUtil.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/EventLoopUtil.java @@ -53,12 +53,11 @@ public static EventLoopGroup newEventLoopGroup(int nThreads, boolean enableBusyW } else if (!enableBusyWait) { return new EpollEventLoopGroup(nThreads, threadFactory); } else { - EpollEventLoopGroup eventLoopGroup = new EpollEventLoopGroup(nThreads, threadFactory, () -> { + return new EpollEventLoopGroup(nThreads, threadFactory, () -> { return (selectSupplier, hasTasks) -> { return -3; }; }); - return eventLoopGroup; } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/ProxyUtils.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/ProxyUtils.java index b3b455b9d33..b7bd42ab2b5 100644 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/ProxyUtils.java +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/ProxyUtils.java @@ -20,8 +20,8 @@ import org.apache.inlong.common.msg.AttributeConstants; import org.apache.inlong.sdk.dataproxy.ConfigConstants; import org.apache.inlong.sdk.dataproxy.ProxyClientConfig; -import org.apache.inlong.sdk.dataproxy.network.Utils; +import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -106,7 +106,8 @@ public static boolean isBodyLengthValid(byte[] body, int maxLen) { } // Reserve space for attribute if (body.length > maxLen - ConfigConstants.RESERVED_ATTRIBUTE_LENGTH) { - logger.debug("body length is too long, max length is {}", maxLen); + logger.debug("body length({}) > max length({}) - fixed attribute length({})", + body.length, maxLen, ConfigConstants.RESERVED_ATTRIBUTE_LENGTH); return false; } return true; @@ -126,11 +127,12 @@ public static boolean isBodyLengthValid(List bodyList, int maxLen) { int size = 0; for (byte[] body : bodyList) { size += body.length; - // Reserve space for attribute - if (size > maxLen - ConfigConstants.RESERVED_ATTRIBUTE_LENGTH) { - logger.debug("body length is too long, max length is {}", maxLen); - return false; - } + } + // Reserve space for attribute + if (size > maxLen - ConfigConstants.RESERVED_ATTRIBUTE_LENGTH) { + logger.debug("bodyList length({}) > max length({}) - fixed attribute length({})", + size, maxLen, ConfigConstants.RESERVED_ATTRIBUTE_LENGTH); + return false; } return true; } @@ -148,11 +150,11 @@ public static long covertZeroDt(long dt) { * @param clientConfig */ public static void validClientConfig(ProxyClientConfig clientConfig) { - if (clientConfig.isNeedAuthentication()) { - if (Utils.isBlank(clientConfig.getUserName())) { - throw new IllegalArgumentException("Authentication require userName not Blank!"); + if (clientConfig.isEnableAuthentication()) { + if (StringUtils.isBlank(clientConfig.getAuthSecretId())) { + throw new IllegalArgumentException("Authentication require secretId not Blank!"); } - if (Utils.isBlank(clientConfig.getSecretKey())) { + if (StringUtils.isBlank(clientConfig.getAuthSecretKey())) { throw new IllegalArgumentException("Authentication require secretKey not Blank!"); } } diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/ServiceDiscoveryUtils.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/ServiceDiscoveryUtils.java deleted file mode 100644 index 8287608549e..00000000000 --- a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/ServiceDiscoveryUtils.java +++ /dev/null @@ -1,279 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.inlong.sdk.dataproxy.utils; - -import org.apache.inlong.common.util.BasicAuth; -import org.apache.inlong.sdk.dataproxy.ProxyClientConfig; -import org.apache.inlong.sdk.dataproxy.network.Utils; - -import com.google.gson.JsonArray; -import com.google.gson.JsonElement; -import com.google.gson.JsonObject; -import com.google.gson.JsonParser; -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.http.Header; -import org.apache.http.HttpResponse; -import org.apache.http.client.config.RequestConfig; -import org.apache.http.client.entity.UrlEncodedFormEntity; -import org.apache.http.client.methods.HttpPost; -import org.apache.http.conn.ssl.SSLConnectionSocketFactory; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.DefaultHttpClient; -import org.apache.http.impl.client.HttpClients; -import org.apache.http.message.BasicHeader; -import org.apache.http.message.BasicNameValuePair; -import org.apache.http.params.BasicHttpParams; -import org.apache.http.params.HttpConnectionParams; -import org.apache.http.params.HttpParams; -import org.apache.http.ssl.SSLContexts; -import org.apache.http.util.EntityUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.net.ssl.SSLContext; - -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; -import java.nio.charset.StandardCharsets; -import java.security.SecureRandom; -import java.util.ArrayList; -import java.util.List; - -/** - * Utils for service discovery - */ -public class ServiceDiscoveryUtils { - - private static final Logger log = LoggerFactory.getLogger(ServiceDiscoveryUtils.class); - - private static final String GET_MANAGER_IP_LIST_API = "/inlong/manager/openapi/agent/getManagerIpList"; - private static String latestManagerIPList = ""; - private static String arraySed = ","; - - /** - * Get Inlong-Manager IP list from the given proxy client config - */ - public static String getManagerIpList(ProxyClientConfig clientConfig) { - String managerAddress = clientConfig.getManagerAddress(); - if (StringUtils.isBlank(managerAddress)) { - log.error("ServiceDiscovery get managerIpList but managerAddress is blank, just return"); - return null; - } - - String managerIpList = getManagerIpListByHttp(managerAddress, clientConfig); - if (StringUtils.isNotBlank(managerIpList)) { - latestManagerIPList = managerIpList; - return managerIpList; - } - - log.error("ServiceDiscovery get managerIpList from {} occur error, try to get from latestManagerIPList", - managerAddress); - - String[] managerIps = latestManagerIPList.split(arraySed); - if (managerIps.length > 0) { - for (String managerIp : managerIps) { - if (StringUtils.isBlank(managerIp)) { - log.error("ServiceDiscovery managerIp is null, latestManagerIPList is {}", latestManagerIPList); - continue; - } - - String currentAddress = managerIp + ":" + clientConfig.getManagerPort(); - managerIpList = getManagerIpListByHttp(currentAddress, clientConfig); - if (StringUtils.isBlank(managerIpList)) { - log.error("ServiceDiscovery get latestManagerIPList from {} but got nothing, will try next ip", - managerIp); - continue; - } - latestManagerIPList = managerIpList; - return managerIpList; - } - } else { - log.error("ServiceDiscovery latestManagerIpList {} format error, or not contain ip", latestManagerIPList); - } - - String existedIpList = getLocalManagerIpList(clientConfig.getManagerIpLocalPath()); - if (StringUtils.isNotBlank(existedIpList)) { - String[] existedIps = existedIpList.split(arraySed); - if (existedIps.length > 0) { - for (String existedIp : existedIps) { - if (StringUtils.isBlank(existedIp)) { - log.error("ServiceDiscovery get illegal format ipList from local file, " - + "exist ip is empty, managerIpList is {}, local file is {}", - existedIpList, clientConfig.getManagerIpLocalPath()); - continue; - } - - String currentAddress = existedIp + ":" + clientConfig.getManagerPort(); - managerIpList = getManagerIpListByHttp(currentAddress, clientConfig); - if (StringUtils.isBlank(managerIpList)) { - log.error("ServiceDiscovery get {} from local file {} but got nothing, will try next ip", - existedIp, clientConfig.getManagerIpLocalPath()); - continue; - } - latestManagerIPList = managerIpList; - return managerIpList; - } - } else { - log.error("ServiceDiscovery get illegal format ipList from local file, " - + "exist ip is empty, managerIpList is {}, local file is {}", - existedIpList, clientConfig.getManagerIpLocalPath()); - } - } else { - log.error("ServiceDiscovery get empty ipList from local file {}", clientConfig.getManagerIpLocalPath()); - } - - return managerIpList; - } - - /** - * Get Inlong-Manager IP list from the given managerIp and proxy client config - */ - public static String getManagerIpListByHttp(String managerIp, ProxyClientConfig proxyClientConfig) { - String url = managerIp + GET_MANAGER_IP_LIST_API; - ArrayList params = new ArrayList(); - params.add(new BasicNameValuePair("operation", "query")); - params.add(new BasicNameValuePair("username", proxyClientConfig.getUserName())); - - log.info("Begin to get configure from manager {}, param is {}", url, params); - CloseableHttpClient httpClient; - HttpParams myParams = new BasicHttpParams(); - HttpConnectionParams.setConnectionTimeout(myParams, proxyClientConfig.getManagerConnectionTimeout()); - HttpConnectionParams.setSoTimeout(myParams, proxyClientConfig.getManagerSocketTimeout()); - if (proxyClientConfig.isRequestByHttp()) { - httpClient = new DefaultHttpClient(myParams); - } else { - try { - ArrayList
headers = new ArrayList<>(); - for (BasicNameValuePair paramItem : params) { - headers.add(new BasicHeader(paramItem.getName(), paramItem.getValue())); - } - RequestConfig requestConfig = RequestConfig.custom() - .setConnectTimeout(10000).setSocketTimeout(30000).build(); - SSLContext sslContext = SSLContexts.custom().build(); - SSLConnectionSocketFactory sslsf = new SSLConnectionSocketFactory(sslContext, - new String[]{"TLSv1"}, null, - SSLConnectionSocketFactory.getDefaultHostnameVerifier()); - httpClient = HttpClients.custom().setDefaultHeaders(headers) - .setDefaultRequestConfig(requestConfig).setSSLSocketFactory(sslsf).build(); - } catch (Throwable t) { - log.error("Create Https client failed: ", t); - return null; - } - } - - HttpPost httpPost = null; - try { - httpPost = new HttpPost(url); - if (proxyClientConfig.isNeedAuthentication()) { - long timestamp = System.currentTimeMillis(); - int nonce = new SecureRandom(String.valueOf(timestamp).getBytes()).nextInt(Integer.MAX_VALUE); - httpPost.setHeader(BasicAuth.BASIC_AUTH_HEADER, - Utils.getAuthorizenInfo(proxyClientConfig.getUserName(), - proxyClientConfig.getSecretKey(), timestamp, nonce)); - } - httpPost.setEntity(new UrlEncodedFormEntity(params)); - HttpResponse response = httpClient.execute(httpPost); - String returnStr = EntityUtils.toString(response.getEntity()); - if (Utils.isNotBlank(returnStr) && response.getStatusLine().getStatusCode() == 200) { - log.info("Get configure from manager is " + returnStr); - JsonParser jsonParser = new JsonParser(); - JsonObject jb = jsonParser.parse(returnStr).getAsJsonObject(); - if (jb == null) { - log.warn("ServiceDiscovery updated manager ip failed, returnStr = {} jb is " - + "null ", returnStr, jb); - return null; - } - JsonArray retData = jb.get("data").getAsJsonArray(); - List managerIpList = new ArrayList<>(); - for (JsonElement datum : retData) { - JsonObject record = datum.getAsJsonObject(); - managerIpList.add(record.get("ip").getAsString()); - } - if (managerIpList.isEmpty()) { - return null; - } - String strIPs = String.join(",", managerIpList); - log.info("ServiceDiscovery updated manager ip success, ip : " + strIPs + ", retStr : " + returnStr); - return strIPs; - } - return null; - } catch (Throwable t) { - log.error("Connect Manager error: ", t); - return null; - } finally { - if (httpPost != null) { - httpPost.releaseConnection(); - } - if (httpClient != null) { - httpClient.getConnectionManager().shutdown(); - } - } - } - - /** - * Get Inlong-Manager IP list from local path - */ - public static String getLocalManagerIpList(String localPath) { - log.info("ServiceDiscovery start loading config from file {} ...", localPath); - String newestIp = null; - try { - File managerIpListFile = new File(localPath); - if (!managerIpListFile.exists()) { - log.info("ServiceDiscovery not found local groupIdInfo file from {}", localPath); - return null; - } - byte[] serialized = FileUtils.readFileToByteArray(managerIpListFile); - if (serialized == null) { - return null; - } - newestIp = new String(serialized, StandardCharsets.UTF_8); - log.info("ServiceDiscovery get manager ip list from local success, result is: {}", newestIp); - } catch (IOException e) { - log.error("ServiceDiscovery load manager config error: ", e); - } - - return newestIp; - } - - /** - * Update Inlong-Manager info to local file - */ - public static void updateManagerInfo2Local(String storeString, String path) { - if (StringUtils.isBlank(storeString)) { - log.warn("ServiceDiscovery updateTdmInfo2Local error, configMap is empty or managerIpList is blank"); - return; - } - File localPath = new File(path); - if (!localPath.getParentFile().exists()) { - localPath.getParentFile().mkdirs(); - } - - try (BufferedWriter writer = new BufferedWriter( - new OutputStreamWriter(new FileOutputStream(localPath), StandardCharsets.UTF_8))) { - writer.write(storeString); - writer.flush(); - } catch (IOException e) { - log.error("ServiceDiscovery save manager config error: ", e); - } - } - -} diff --git a/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/Tuple2.java b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/Tuple2.java new file mode 100644 index 00000000000..e5ba8c2bc22 --- /dev/null +++ b/inlong-sdk/dataproxy-sdk/src/main/java/org/apache/inlong/sdk/dataproxy/utils/Tuple2.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sdk.dataproxy.utils; + +public class Tuple2 { + + /** Field 0 of the tuple. */ + private T0 f0 = null; + /** Field 1 of the tuple. */ + private T1 f1 = null; + + /** + * Creates a new tuple where all fields are null. + */ + public Tuple2() { + + } + + /** + * Creates a new tuple with field 0 specified. + * + * @param value0 The value for field 0 + */ + public Tuple2(T0 value0) { + this.f0 = value0; + } + + /** + * Creates a new tuple and assigns the given values to the tuple's fields. + * + * @param value0 The value for field 0 + * @param value1 The value for field 1 + */ + public Tuple2(T0 value0, T1 value1) { + setF0AndF1(value0, value1); + } + + public T0 getF0() { + return f0; + } + + public T1 getF1() { + return f1; + } + + /** + * Set all field values + * + * @param value0 The value for field 0 + * @param value1 The value for field 1 + */ + public void setF0AndF1(T0 value0, T1 value1) { + this.f0 = value0; + this.f1 = value1; + } +} diff --git a/inlong-sdk/dataproxy-sdk/src/test/java/org/apache/inlong/sdk/dataproxy/ProxyConfigManagerTest.java b/inlong-sdk/dataproxy-sdk/src/test/java/org/apache/inlong/sdk/dataproxy/ProxyConfigManagerTest.java index 75b6230a971..40fec3b57d4 100644 --- a/inlong-sdk/dataproxy-sdk/src/test/java/org/apache/inlong/sdk/dataproxy/ProxyConfigManagerTest.java +++ b/inlong-sdk/dataproxy-sdk/src/test/java/org/apache/inlong/sdk/dataproxy/ProxyConfigManagerTest.java @@ -20,6 +20,7 @@ import org.apache.inlong.sdk.dataproxy.config.ProxyConfigEntry; import org.apache.inlong.sdk.dataproxy.config.ProxyConfigManager; import org.apache.inlong.sdk.dataproxy.network.ClientMgr; +import org.apache.inlong.sdk.dataproxy.utils.Tuple2; import org.junit.Assert; import org.junit.Test; @@ -36,15 +37,18 @@ public class ProxyConfigManagerTest { .toString(); private final ProxyClientConfig clientConfig = PowerMockito.mock(ProxyClientConfig.class); private final ClientMgr clientMgr = PowerMockito.mock(ClientMgr.class); - private final ProxyConfigManager proxyConfigManager = new ProxyConfigManager(clientConfig, "127.0.0.1", - clientMgr); + private final ProxyConfigManager proxyConfigManager; public ProxyConfigManagerTest() throws URISyntaxException { + clientConfig.setConfigStoreBasePath(localFile); + proxyConfigManager = + new ProxyConfigManager("test", clientConfig, clientMgr); } @Test public void testProxyConfigParse() throws Exception { - ProxyConfigEntry proxyEntry = proxyConfigManager.getLocalProxyListFromFile(localFile); + Tuple2 result = proxyConfigManager.getLocalProxyListFromFile(localFile); + ProxyConfigEntry proxyEntry = result.getF0(); Assert.assertEquals(proxyEntry.isInterVisit(), false); Assert.assertEquals(proxyEntry.getLoad(), 12); Assert.assertEquals(proxyEntry.getClusterId(), 1); diff --git a/inlong-sdk/dataproxy-sdk/src/test/java/org/apache/inlong/sdk/dataproxy/UtilsTest.java b/inlong-sdk/dataproxy-sdk/src/test/java/org/apache/inlong/sdk/dataproxy/UtilsTest.java index 434e3410dfc..b1a0ccd7bb3 100644 --- a/inlong-sdk/dataproxy-sdk/src/test/java/org/apache/inlong/sdk/dataproxy/UtilsTest.java +++ b/inlong-sdk/dataproxy-sdk/src/test/java/org/apache/inlong/sdk/dataproxy/UtilsTest.java @@ -17,7 +17,7 @@ package org.apache.inlong.sdk.dataproxy; -import org.apache.inlong.sdk.dataproxy.network.Utils; +import org.apache.inlong.sdk.dataproxy.network.IpUtils; import org.junit.Assert; import org.junit.Test; @@ -26,7 +26,7 @@ public class UtilsTest { @Test public void getLocalIp() { - String ip = Utils.getLocalIp(); + String ip = IpUtils.getLocalIp(); Assert.assertNotNull(ip); } diff --git a/inlong-sdk/dirty-data-sdk/src/main/java/org/apache/inlong/sdk/dirtydata/DirtyMessageWrapper.java b/inlong-sdk/dirty-data-sdk/src/main/java/org/apache/inlong/sdk/dirtydata/DirtyMessageWrapper.java index a82d574cacf..a22c4249b03 100644 --- a/inlong-sdk/dirty-data-sdk/src/main/java/org/apache/inlong/sdk/dirtydata/DirtyMessageWrapper.java +++ b/inlong-sdk/dirty-data-sdk/src/main/java/org/apache/inlong/sdk/dirtydata/DirtyMessageWrapper.java @@ -34,7 +34,8 @@ public class DirtyMessageWrapper { private static DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss"); - private String delimiter; + @Builder.Default + private String delimiter = "|"; @Builder.Default @Getter private int retryTimes = 0; diff --git a/inlong-sdk/dirty-data-sdk/src/main/java/org/apache/inlong/sdk/dirtydata/InlongSdkDirtySender.java b/inlong-sdk/dirty-data-sdk/src/main/java/org/apache/inlong/sdk/dirtydata/InlongSdkDirtySender.java index 80cc596c265..1965ef37e34 100644 --- a/inlong-sdk/dirty-data-sdk/src/main/java/org/apache/inlong/sdk/dirtydata/InlongSdkDirtySender.java +++ b/inlong-sdk/dirty-data-sdk/src/main/java/org/apache/inlong/sdk/dirtydata/InlongSdkDirtySender.java @@ -31,7 +31,6 @@ import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; @Slf4j @Builder @@ -63,7 +62,7 @@ public void init() throws Exception { ProxyClientConfig proxyClientConfig = new ProxyClientConfig(InetAddress.getLocalHost().getHostAddress(), true, inlongManagerAddr, inlongManagerPort, inlongGroupId, authId, authKey); - proxyClientConfig.setReadProxyIPFromLocal(false); + proxyClientConfig.setOnlyUseLocalProxyConfig(false); proxyClientConfig.setAsyncCallbackSize(maxCallbackSize); this.sender = DefaultMessageSender.generateSenderByClusterId(proxyClientConfig); this.sender.setMsgtype(7); @@ -75,7 +74,11 @@ public void init() throws Exception { } public void sendDirtyMessage(DirtyMessageWrapper messageWrapper) throws InterruptedException { - dirtyDataQueue.offer(messageWrapper, 10, TimeUnit.SECONDS); + dirtyDataQueue.put(messageWrapper); + } + + public boolean sendDirtyMessageAsync(DirtyMessageWrapper messageWrapper) { + return dirtyDataQueue.offer(messageWrapper); } private void doSendDirtyMessage() { diff --git a/inlong-sort-standalone/sort-standalone-source/src/main/java/org/apache/inlong/sort/standalone/sink/kafka/KafkaIdConfig.java b/inlong-sort-standalone/sort-standalone-source/src/main/java/org/apache/inlong/sort/standalone/sink/kafka/KafkaIdConfig.java index a1c01bdb730..c6d29a9c6d8 100644 --- a/inlong-sort-standalone/sort-standalone-source/src/main/java/org/apache/inlong/sort/standalone/sink/kafka/KafkaIdConfig.java +++ b/inlong-sort-standalone/sort-standalone-source/src/main/java/org/apache/inlong/sort/standalone/sink/kafka/KafkaIdConfig.java @@ -21,6 +21,7 @@ import org.apache.inlong.common.pojo.sort.dataflow.DataFlowConfig; import org.apache.inlong.common.pojo.sort.dataflow.dataType.CsvConfig; import org.apache.inlong.common.pojo.sort.dataflow.dataType.DataTypeConfig; +import org.apache.inlong.common.pojo.sort.dataflow.dataType.KvConfig; import org.apache.inlong.common.pojo.sort.dataflow.sink.KafkaSinkConfig; import org.apache.inlong.sort.standalone.config.pojo.IdConfig; import org.apache.inlong.sort.standalone.config.pojo.InlongId; @@ -66,6 +67,8 @@ public static KafkaIdConfig create(DataFlowConfig dataFlowConfig) { String separator = DEFAULT_SEPARATOR; if (dataTypeConfig instanceof CsvConfig) { separator = String.valueOf(((CsvConfig) dataTypeConfig).getDelimiter()); + } else if (dataTypeConfig instanceof KvConfig) { + separator = String.valueOf(((KvConfig) dataTypeConfig).getEntrySplitter()); } return KafkaIdConfig.builder() diff --git a/inlong-sort/sort-core/pom.xml b/inlong-sort/sort-core/pom.xml index 7d20ed65a8b..74f0a1e3eb0 100644 --- a/inlong-sort/sort-core/pom.xml +++ b/inlong-sort/sort-core/pom.xml @@ -430,6 +430,12 @@ ${project.version} test + + org.apache.inlong + sort-connector-kafka-v1.18 + ${project.version} + test + diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/pom.xml b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/pom.xml index 78e19a16eba..850d5dddfda 100644 --- a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/pom.xml +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/pom.xml @@ -31,8 +31,8 @@ ${project.parent.parent.parent.basedir} 1.18.1 - 6.8.17 2.15.3-18.0 + 3.7.1 @@ -51,6 +51,10 @@ postgresql ${testcontainers.version} + + org.testcontainers + kafka + org.postgresql postgresql @@ -61,16 +65,11 @@ elasticsearch ${testcontainers.version} - - org.elasticsearch.client - elasticsearch-rest-high-level-client - ${elasticsearch.version} - org.elasticsearch.client elasticsearch-rest-client - ${elasticsearch.version} + ${elasticsearch7.version} @@ -142,6 +141,17 @@ ${flink.version} test + + org.apache.kafka + kafka-clients + ${kafka.clients.version} + test + + + co.elastic.clients + elasticsearch-java + ${elasticsearch7.version} + @@ -158,6 +168,23 @@ jar ${project.build.directory}/dependencies + + org.apache.inlong + sort-connector-kafka-v1.18 + ${project.version} + sort-connector-kafka.jar + jar + ${project.build.directory}/dependencies + + + org.apache.inlong + sort-connector-elasticsearch7-v1.18 + ${project.version} + sort-connector-elasticsearch7.jar + jar + ${project.build.directory}/dependencies + + @@ -203,6 +230,11 @@ org.apache.maven.plugins maven-surefire-plugin ${plugin.surefire.version} + + + src/test/resources/log4j2-test.properties + + diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/Kafka2Elasticsearch7Test.java b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/Kafka2Elasticsearch7Test.java new file mode 100644 index 00000000000..0cf94a06632 --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/java/org/apache/inlong/sort/tests/Kafka2Elasticsearch7Test.java @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.tests; + +import org.apache.inlong.sort.tests.utils.FlinkContainerTestEnvJRE8; +import org.apache.inlong.sort.tests.utils.PlaceholderResolver; +import org.apache.inlong.sort.tests.utils.TestUtils; + +import co.elastic.clients.elasticsearch.ElasticsearchClient; +import co.elastic.clients.json.jackson.JacksonJsonpMapper; +import co.elastic.clients.transport.rest_client.RestClientTransport; +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestClient; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container.ExecResult; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.elasticsearch.ElasticsearchContainer; +import org.testcontainers.utility.DockerImageName; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +public class Kafka2Elasticsearch7Test extends FlinkContainerTestEnvJRE8 { + + private static final Logger LOG = LoggerFactory.getLogger(Kafka2Elasticsearch7Test.class); + public static final Logger KAFKA_LOG = LoggerFactory.getLogger(KafkaContainer.class); + public static final Logger ELASTICSEARCH_LOGGER = LoggerFactory.getLogger(ElasticsearchContainer.class); + + private static final Path kafkaJar = TestUtils.getResource("sort-connector-kafka.jar"); + private static final Path elasticsearchJar = TestUtils.getResource("sort-connector-elasticsearch7.jar"); + + private static final int ELASTICSEARCH_DEFAULT_PORT = 9200; + + private static final String FIRST_KAFKA_MESSAGE = "{\"message\":\"Hello From Kafka\"}"; + private static final String SECOND_KAFKA_MESSAGE = "{\"message\":\"Goodbye From ElasticSearch\"}"; + + private static final String FIRST_EXPECTED_MESSAGE = "Hello From Kafka"; + private static final String SECOND_EXPECTED_MESSAGE = "Goodbye From ElasticSearch"; + + private static final String sqlFile; + + static { + try { + sqlFile = Paths + .get(Kafka2Elasticsearch7Test.class.getResource("/flinkSql/kafka_to_elasticsearch.sql").toURI()) + .toString(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @ClassRule + public static final KafkaContainer KAFKA = + new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.2.1")) + .withNetwork(NETWORK) + .withNetworkAliases("kafka") + .withEmbeddedZookeeper() + .withLogConsumer(new Slf4jLogConsumer(KAFKA_LOG)); + + @ClassRule + public static final ElasticsearchContainer ELASTICSEARCH = + new ElasticsearchContainer(DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch:7.17.13")) + .withNetwork(NETWORK) + .withNetworkAliases("elasticsearch") + .withLogConsumer(new Slf4jLogConsumer(ELASTICSEARCH_LOGGER)); + + @Before + public void setup() throws IOException { + waitUntilJobRunning(Duration.ofSeconds(30)); + initializeKafkaTopic("test-topic"); + initializeElasticsearchIndex(); + } + + private void initializeKafkaTopic(String topic) { + String fileName = "kafka_test_kafka_init.txt"; + int port = KafkaContainer.ZOOKEEPER_PORT; + + Map properties = new HashMap<>(); + properties.put("TOPIC", topic); + properties.put("ZOOKEEPER_PORT", port); + + try { + String createKafkaStatement = getCreateStatement(fileName, properties); + ExecResult result = KAFKA.execInContainer("bash", "-c", createKafkaStatement); + LOG.info("Create kafka topic: {}, std: {}", createKafkaStatement, result.getStdout()); + if (result.getExitCode() != 0) { + throw new RuntimeException("Init kafka topic failed. Exit code:" + result.getExitCode()); + } + } catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + private String getCreateStatement(String fileName, Map properties) { + URL url = Objects.requireNonNull(Kafka2Elasticsearch7Test.class.getResource("/env/" + fileName)); + + try { + Path file = Paths.get(url.toURI()); + return PlaceholderResolver.getDefaultResolver().resolveByMap( + new String(Files.readAllBytes(file), StandardCharsets.UTF_8), + properties); + } catch (IOException | URISyntaxException e) { + throw new RuntimeException(e); + } + } + + private void initializeElasticsearchIndex() throws IOException { + RestClient restClient = RestClient.builder( + new HttpHost("localhost", ELASTICSEARCH.getMappedPort(ELASTICSEARCH_DEFAULT_PORT), "http")) + .build(); + RestClientTransport transport = new RestClientTransport(restClient, new JacksonJsonpMapper()); + ElasticsearchClient client = new ElasticsearchClient(transport); + + client.indices().create(c -> c.index("test-index")); + LOG.info("Created Elasticsearch index: test-index"); + } + + @AfterClass + public static void teardown() { + if (KAFKA != null) { + KAFKA.stop(); + } + if (ELASTICSEARCH != null) { + ELASTICSEARCH.stop(); + } + } + + @Test + public void testKafkaToElasticsearch() throws Exception { + submitSQLJob(sqlFile, kafkaJar, elasticsearchJar); + waitUntilJobRunning(Duration.ofSeconds(10)); + + // Produce messages to Kafka + org.apache.kafka.clients.producer.KafkaProducer producer = + new org.apache.kafka.clients.producer.KafkaProducer<>(getKafkaProducerConfig()); + producer.send( + new org.apache.kafka.clients.producer.ProducerRecord<>("test-topic", "key1", FIRST_KAFKA_MESSAGE)); + producer.send( + new org.apache.kafka.clients.producer.ProducerRecord<>("test-topic", "key2", SECOND_KAFKA_MESSAGE)); + + // Query Elasticsearch to verify data is ingested + RestClient restClient = RestClient.builder( + new HttpHost("localhost", ELASTICSEARCH.getMappedPort(9200), "http")) + .build(); + RestClientTransport transport = new RestClientTransport(restClient, new JacksonJsonpMapper()); + ElasticsearchClient client = new ElasticsearchClient(transport); + + List messages = new ArrayList<>(); + int maxRetries = 10; // Maximum number of retries (10 seconds) + int retryCount = 0; + + while (retryCount < maxRetries) { + co.elastic.clients.elasticsearch.core.SearchRequest searchRequest = + new co.elastic.clients.elasticsearch.core.SearchRequest.Builder() + .index("test-index") + .query(q -> q.matchAll(m -> m)) + .build(); + + co.elastic.clients.elasticsearch.core.SearchResponse response = + client.search(searchRequest, Map.class); + + // Extract `message` fields using Elasticsearch Java API + messages = response.hits().hits().stream() + .map(hit -> { + @SuppressWarnings("unchecked") + Map source = hit.source(); + if (source != null && source.containsKey("message")) { + return (String) source.get("message"); + } + return null; + }) + .filter(Objects::nonNull) // Remove null values + .collect(Collectors.toList()); + + if (!messages.isEmpty()) { + // Stop polling if data is found + break; + } + + // Wait for 1 second before retrying + Thread.sleep(1000); + retryCount++; + } + + if (messages.isEmpty()) { + throw new AssertionError("Elasticsearch validation failed: No messages found after polling."); + } + + LOG.info("Extracted messages from Elasticsearch: {}", messages); + + // Create expected messages list + List expectedMessages = new ArrayList<>(); + expectedMessages.add(FIRST_EXPECTED_MESSAGE); + expectedMessages.add(SECOND_EXPECTED_MESSAGE); + + // Validate messages against the expected messages + if (new HashSet<>(messages).equals(new HashSet<>(expectedMessages))) { + LOG.info("Elasticsearch contains all expected messages: {}", expectedMessages); + } else { + throw new AssertionError( + String.format("Elasticsearch validation failed. Expected: %s, Found: %s", expectedMessages, + messages)); + } + } + + private java.util.Properties getKafkaProducerConfig() { + java.util.Properties props = new java.util.Properties(); + String bootstrapServers = KAFKA.getBootstrapServers(); + props.put("bootstrap.servers", bootstrapServers); + props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + return props; + } +} diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/env/kafka_test_kafka_init.txt b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/env/kafka_test_kafka_init.txt new file mode 100644 index 00000000000..b2f31d78fa4 --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/env/kafka_test_kafka_init.txt @@ -0,0 +1 @@ +kafka-topics --create --topic ${TOPIC} --replication-factor 1 --partitions 1 --zookeeper localhost:${ZOOKEEPER_PORT} \ No newline at end of file diff --git a/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/flinkSql/kafka_to_elasticsearch.sql b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/flinkSql/kafka_to_elasticsearch.sql new file mode 100644 index 00000000000..77cdeb8caef --- /dev/null +++ b/inlong-sort/sort-end-to-end-tests/sort-end-to-end-tests-v1.18/src/test/resources/flinkSql/kafka_to_elasticsearch.sql @@ -0,0 +1,24 @@ +CREATE TABLE kafka_source ( + `message` STRING +) WITH ( + 'connector' = 'kafka-inlong', + 'topic' = 'test-topic', + 'properties.bootstrap.servers' = 'kafka:9092', + 'properties.group.id' = 'flink-group', + 'scan.startup.mode' = 'earliest-offset', + 'format' = 'json' +); + + +CREATE TABLE elasticsearch_sink ( + `message` STRING +) WITH ( + 'connector' = 'elasticsearch7-inlong', + 'hosts' = 'http://elasticsearch:9200', + 'index' = 'test-index', + 'format' = 'json' +); + + +INSERT INTO elasticsearch_sink +SELECT * FROM kafka_source; diff --git a/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/DirtyData.java b/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/DirtyData.java index 24c5dddecd0..76708799258 100644 --- a/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/DirtyData.java +++ b/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/DirtyData.java @@ -17,7 +17,6 @@ package org.apache.inlong.sort.base.dirty; -import org.apache.inlong.sort.base.dirty.sink.DirtyServerType; import org.apache.inlong.sort.base.util.PatternReplaceUtils; import org.apache.flink.table.types.logical.LogicalType; @@ -65,7 +64,7 @@ public class DirtyData { */ private final DirtyType dirtyType; - private final DirtyServerType serverType; + private final String serverType; /** * Dirty describe message, it is the cause of dirty data */ @@ -88,7 +87,7 @@ public class DirtyData { private final T data; public DirtyData(T data, String identifier, String labels, - String logTag, DirtyType dirtyType, DirtyServerType serverType, String dirtyMessage, + String logTag, DirtyType dirtyType, String serverType, String dirtyMessage, @Nullable LogicalType rowType, long dataTime, String extParams) { this.data = data; this.dirtyType = dirtyType; @@ -131,7 +130,7 @@ public DirtyType getDirtyType() { return dirtyType; } - public DirtyServerType getServerType() { + public String getServerType() { return serverType; } @@ -162,7 +161,7 @@ public static class Builder { private String labels; private String logTag; private DirtyType dirtyType = DirtyType.UNDEFINED; - private DirtyServerType serverType = DirtyServerType.UNDEFINED; + private String serverType; private String dirtyMessage; private LogicalType rowType; private long dataTime; @@ -184,7 +183,7 @@ public Builder setDirtyType(DirtyType dirtyType) { return this; } - public Builder setServerType(DirtyServerType serverType) { + public Builder setServerType(String serverType) { this.serverType = serverType; return this; } diff --git a/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/sink/sdk/InlongSdkDirtySink.java b/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/sink/sdk/InlongSdkDirtySink.java index 8513f841bcd..daec1c0694e 100644 --- a/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/sink/sdk/InlongSdkDirtySink.java +++ b/inlong-sort/sort-flink/base/src/main/java/org/apache/inlong/sort/base/dirty/sink/sdk/InlongSdkDirtySink.java @@ -67,14 +67,14 @@ public void invoke(DirtyData dirtyData) throws Exception { .inlongStreamId(dataStreamId) .dataflowId(dataflowId) .dataTime(dirtyData.getDataTime()) - .serverType(dirtyData.getServerType().format()) + .serverType(dirtyData.getServerType()) .dirtyType(dirtyData.getDirtyType().format()) .dirtyMessage(dirtyData.getDirtyMessage()) .ext(dirtyData.getExtParams()) .data(dirtyMessage) .build(); - dirtySender.sendDirtyMessage(wrapper); + dirtySender.sendDirtyMessageAsync(wrapper); } catch (Throwable t) { log.error("failed to send dirty message to inlong sdk", t); if (!options.isIgnoreSideOutputErrors()) { diff --git a/inlong-sort/sort-flink/sort-flink-v1.13/sort-connectors/tubemq/src/main/java/org/apache/inlong/sort/tubemq/FlinkTubeMQConsumer.java b/inlong-sort/sort-flink/sort-flink-v1.13/sort-connectors/tubemq/src/main/java/org/apache/inlong/sort/tubemq/FlinkTubeMQConsumer.java index abd69f8ecb7..4cbd285f175 100644 --- a/inlong-sort/sort-flink/sort-flink-v1.13/sort-connectors/tubemq/src/main/java/org/apache/inlong/sort/tubemq/FlinkTubeMQConsumer.java +++ b/inlong-sort/sort-flink/sort-flink-v1.13/sort-connectors/tubemq/src/main/java/org/apache/inlong/sort/tubemq/FlinkTubeMQConsumer.java @@ -69,6 +69,8 @@ public class FlinkTubeMQConsumer extends RichParallelSourceFunction private static final Logger LOG = LoggerFactory.getLogger(FlinkTubeMQConsumer.class); private static final String TUBE_OFFSET_STATE = "tube-offset-state"; + private static final String UNDERSCORE = "_"; + /** * The address of TubeMQ master, format eg: 127.0.0.1:8715,127.0.0.2:8715. */ @@ -221,7 +223,10 @@ public void open(Configuration parameters) throws Exception { messagePullConsumer = messageSessionFactory.createPullConsumer(consumerConfig); messagePullConsumer.subscribe(topic, streamIdSet); String jobId = getRuntimeContext().getJobId().toString(); - messagePullConsumer.completeSubscribe(sessionKey.concat(jobId), numTasks, true, currentOffsets); + String attemptNumber = String.valueOf(getRuntimeContext().getAttemptNumber()); + String startSessionKey = sessionKey.concat(UNDERSCORE).concat(jobId).concat(UNDERSCORE).concat(attemptNumber); + LOG.info("start to init tube mq consumer, session key={}", startSessionKey); + messagePullConsumer.completeSubscribe(startSessionKey, numTasks, true, currentOffsets); running = true; } diff --git a/inlong-sort/sort-flink/sort-flink-v1.15/sort-connectors/tubemq/src/main/java/org/apache/inlong/sort/tubemq/table/DynamicTubeMQTableDeserializationSchema.java b/inlong-sort/sort-flink/sort-flink-v1.15/sort-connectors/tubemq/src/main/java/org/apache/inlong/sort/tubemq/table/DynamicTubeMQTableDeserializationSchema.java index 94631e7cd39..2749213096d 100644 --- a/inlong-sort/sort-flink/sort-flink-v1.15/sort-connectors/tubemq/src/main/java/org/apache/inlong/sort/tubemq/table/DynamicTubeMQTableDeserializationSchema.java +++ b/inlong-sort/sort-flink/sort-flink-v1.15/sort-connectors/tubemq/src/main/java/org/apache/inlong/sort/tubemq/table/DynamicTubeMQTableDeserializationSchema.java @@ -17,10 +17,10 @@ package org.apache.inlong.sort.tubemq.table; +import org.apache.inlong.common.constant.MQType; import org.apache.inlong.sort.base.dirty.DirtyData; import org.apache.inlong.sort.base.dirty.DirtyOptions; import org.apache.inlong.sort.base.dirty.DirtyType; -import org.apache.inlong.sort.base.dirty.sink.DirtyServerType; import org.apache.inlong.sort.base.dirty.sink.DirtySink; import org.apache.inlong.sort.base.metric.MetricOption; import org.apache.inlong.sort.base.metric.MetricsCollector; @@ -144,7 +144,7 @@ public void deserialize(Message message, Collector out) throws IOExcept builder.setData(message.getData()) .setDirtyType(DirtyType.KEY_DESERIALIZE_ERROR) - .setServerType(DirtyServerType.TUBE_MQ) + .setServerType(MQType.TUBEMQ) .setDirtyDataTime(dataTime) .setExtParams(message.getAttribute()) .setLabels(dirtyOptions.getLabels()) diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/pom.xml b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/pom.xml new file mode 100644 index 00000000000..1028cec4d4f --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/pom.xml @@ -0,0 +1,126 @@ + + + + 4.0.0 + + org.apache.inlong + sort-connectors-v1.18 + 2.1.0-SNAPSHOT + + + sort-connector-kafka-v1.18 + jar + Apache InLong - Sort-connector-kafka + + + ${project.parent.parent.parent.parent.parent.basedir} + 3.2.0-1.18 + 3.4.0 + + + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + org.apache.inlong + sort-common + ${project.version} + + + org.apache.inlong + sort-connector-base + ${project.version} + + + org.apache.flink + flink-connector-base + ${flink.version} + + + org.apache.flink + flink-connector-kafka + ${flink.connector.kafka.version} + + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-flink + + shade + + package + + true + + + org.apache.inlong:* + io.streamnative.connectors:kafka-flink-connector-origin* + io.streamnative.connectors:flink-protobuf + org.apache.kafka:* + org.apache.flink:flink-connector-kafka + com.google.protobuf:* + org.bouncycastle*:* + org.bouncycastle*:* + javax.*:* + org.lz4*:* + org.slf4j:jul-to-slf4j + io.airlift:* + + + + + org.apache.inlong:sort-connector-* + + org/apache/inlong/** + META-INF/services/org.apache.flink.table.factories.Factory + + + + *:* + + log4j.properties + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + + + + + diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSource.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSource.java new file mode 100644 index 00000000000..6833e557f26 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSource.java @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.source; + +import org.apache.inlong.sort.kafka.source.reader.KafkaSourceReader; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumState; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumStateSerializer; +import org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.KafkaPartitionSplitReader; +import org.apache.flink.connector.kafka.source.reader.KafkaRecordEmitter; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.connector.kafka.source.reader.fetcher.KafkaSourceFetcherManager; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.util.UserCodeClassLoader; +import org.apache.flink.util.function.SerializableSupplier; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Collection; +import java.util.Optional; +import java.util.Properties; +import java.util.function.Consumer; +import java.util.function.Supplier; + +/** + * The Source implementation of Kafka. Please use a {@link KafkaSourceBuilder} to construct a {@link + * KafkaSource}. The following example shows how to create a KafkaSource emitting records of + * String type. + * + *
{@code
+ * KafkaSource source = KafkaSource
+ *     .builder()
+ *     .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
+ *     .setGroupId("MyGroup")
+ *     .setTopics(Arrays.asList(TOPIC1, TOPIC2))
+ *     .setDeserializer(new TestingKafkaRecordDeserializationSchema())
+ *     .setStartingOffsets(OffsetsInitializer.earliest())
+ *     .build();
+ * }
+ * + *

{@link org.apache.flink.connector.kafka.source.enumerator.KafkaSourceEnumerator} only supports + * adding new splits and not removing splits in split discovery. + * + *

See {@link KafkaSourceBuilder} for more details on how to configure this source. + * + * @param the output type of the source. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +// TODO: Add a variable metricSchema to report audit information +public class KafkaSource + implements + Source, + ResultTypeQueryable { + + private static final long serialVersionUID = -8755372893283732098L; + // Users can choose only one of the following ways to specify the topics to consume from. + private final KafkaSubscriber subscriber; + // Users can specify the starting / stopping offset initializer. + private final OffsetsInitializer startingOffsetsInitializer; + private final OffsetsInitializer stoppingOffsetsInitializer; + // Boundedness + private final Boundedness boundedness; + private final KafkaRecordDeserializationSchema deserializationSchema; + // The configurations. + private final Properties props; + // Client rackId callback + private final SerializableSupplier rackIdSupplier; + + KafkaSource( + KafkaSubscriber subscriber, + OffsetsInitializer startingOffsetsInitializer, + @Nullable OffsetsInitializer stoppingOffsetsInitializer, + Boundedness boundedness, + KafkaRecordDeserializationSchema deserializationSchema, + Properties props, + SerializableSupplier rackIdSupplier) { + this.subscriber = subscriber; + this.startingOffsetsInitializer = startingOffsetsInitializer; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + this.boundedness = boundedness; + this.deserializationSchema = deserializationSchema; + this.props = props; + this.rackIdSupplier = rackIdSupplier; + } + + /** + * Get a kafkaSourceBuilder to build a {@link KafkaSource}. + * + * @return a Kafka source builder. + */ + public static KafkaSourceBuilder builder() { + return new KafkaSourceBuilder<>(); + } + + @Override + public Boundedness getBoundedness() { + return this.boundedness; + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) + throws Exception { + return createReader(readerContext, (ignore) -> { + }); + } + + @VisibleForTesting + SourceReader createReader( + SourceReaderContext readerContext, Consumer> splitFinishedHook) + throws Exception { + FutureCompletingBlockingQueue>> elementsQueue = + new FutureCompletingBlockingQueue<>(); + deserializationSchema.open( + new DeserializationSchema.InitializationContext() { + + @Override + public MetricGroup getMetricGroup() { + return readerContext.metricGroup().addGroup("deserializer"); + } + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + return readerContext.getUserCodeClassLoader(); + } + }); + final KafkaSourceReaderMetrics kafkaSourceReaderMetrics = + new KafkaSourceReaderMetrics(readerContext.metricGroup()); + + Supplier splitReaderSupplier = + () -> new KafkaPartitionSplitReader( + props, + readerContext, + kafkaSourceReaderMetrics, + Optional.ofNullable(rackIdSupplier) + .map(Supplier::get) + .orElse(null)); + KafkaRecordEmitter recordEmitter = new KafkaRecordEmitter<>(deserializationSchema); + + return new KafkaSourceReader<>( + elementsQueue, + new KafkaSourceFetcherManager( + elementsQueue, splitReaderSupplier::get, splitFinishedHook), + recordEmitter, + toConfiguration(props), + readerContext, + kafkaSourceReaderMetrics); + } + + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new KafkaSourceEnumerator( + subscriber, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + props, + enumContext, + boundedness); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, + KafkaSourceEnumState checkpoint) + throws IOException { + return new KafkaSourceEnumerator( + subscriber, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + props, + enumContext, + boundedness, + checkpoint); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new KafkaPartitionSplitSerializer(); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new KafkaSourceEnumStateSerializer(); + } + + @Override + public TypeInformation getProducedType() { + return deserializationSchema.getProducedType(); + } + + // ----------- private helper methods --------------- + + private Configuration toConfiguration(Properties props) { + Configuration config = new Configuration(); + props.stringPropertyNames().forEach(key -> config.setString(key, props.getProperty(key))); + return config; + } + + @VisibleForTesting + Configuration getConfiguration() { + return toConfiguration(props); + } + + @VisibleForTesting + KafkaSubscriber getKafkaSubscriber() { + return subscriber; + } + + @VisibleForTesting + OffsetsInitializer getStoppingOffsetsInitializer() { + return stoppingOffsetsInitializer; + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSourceBuilder.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSourceBuilder.java new file mode 100644 index 00000000000..4cfa147e977 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSourceBuilder.java @@ -0,0 +1,551 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.source; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializerValidator; +import org.apache.flink.connector.kafka.source.enumerator.subscriber.KafkaSubscriber; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.util.function.SerializableSupplier; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.regex.Pattern; + +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * The builder class for {@link KafkaSource} to make it easier for the users to construct a {@link + * KafkaSource}. + * + *

The following example shows the minimum setup to create a KafkaSource that reads the String + * values from a Kafka topic. + * + *

{@code
+ * KafkaSource source = KafkaSource
+ *     .builder()
+ *     .setBootstrapServers(MY_BOOTSTRAP_SERVERS)
+ *     .setTopics(Arrays.asList(TOPIC1, TOPIC2))
+ *     .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class))
+ *     .build();
+ * }
+ * + *

The bootstrap servers, topics/partitions to consume, and the record deserializer are required + * fields that must be set. + * + *

To specify the starting offsets of the KafkaSource, one can call {@link + * #setStartingOffsets(OffsetsInitializer)}. + * + *

By default the KafkaSource runs in an {@link Boundedness#CONTINUOUS_UNBOUNDED} mode and never + * stops until the Flink job is canceled or fails. To let the KafkaSource run as {@link + * Boundedness#CONTINUOUS_UNBOUNDED} yet stop at some given offsets, one can call {@link + * #setUnbounded(OffsetsInitializer)}. For example the following KafkaSource stops after it consumes + * up to the latest partition offsets at the point when the Flink job started. + * + *

{@code
+ * KafkaSource source = KafkaSource
+ *     .builder()
+ *     .setBootstrapServers(MY_BOOTSTRAP_SERVERS)
+ *     .setTopics(Arrays.asList(TOPIC1, TOPIC2))
+ *     .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class))
+ *     .setUnbounded(OffsetsInitializer.latest())
+ *     .setRackId(() -> MY_RACK_ID)
+ *     .build();
+ * }
+ * + *

Check the Java docs of each individual methods to learn more about the settings to build a + * KafkaSource. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +// TODO: Add a variable metricSchema to report audit information +public class KafkaSourceBuilder { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceBuilder.class); + private static final String[] REQUIRED_CONFIGS = {ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG}; + // The subscriber specifies the partitions to subscribe to. + private KafkaSubscriber subscriber; + // Users can specify the starting / stopping offset initializer. + private OffsetsInitializer startingOffsetsInitializer; + private OffsetsInitializer stoppingOffsetsInitializer; + // Boundedness + private Boundedness boundedness; + private KafkaRecordDeserializationSchema deserializationSchema; + // The configurations. + protected Properties props; + // Client rackId supplier + private SerializableSupplier rackIdSupplier; + + KafkaSourceBuilder() { + this.subscriber = null; + this.startingOffsetsInitializer = OffsetsInitializer.earliest(); + this.stoppingOffsetsInitializer = new NoStoppingOffsetsInitializer(); + this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; + this.deserializationSchema = null; + this.props = new Properties(); + this.rackIdSupplier = null; + } + + /** + * Sets the bootstrap servers for the KafkaConsumer of the KafkaSource. + * + * @param bootstrapServers the bootstrap servers of the Kafka cluster. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setBootstrapServers(String bootstrapServers) { + return setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + } + + /** + * Sets the consumer group id of the KafkaSource. + * + * @param groupId the group id of the KafkaSource. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setGroupId(String groupId) { + return setProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId); + } + + /** + * Set a list of topics the KafkaSource should consume from. All the topics in the list should + * have existed in the Kafka cluster. Otherwise an exception will be thrown. To allow some of + * the topics to be created lazily, please use {@link #setTopicPattern(Pattern)} instead. + * + * @param topics the list of topics to consume from. + * @return this KafkaSourceBuilder. + * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Collection) + */ + public KafkaSourceBuilder setTopics(List topics) { + ensureSubscriberIsNull("topics"); + subscriber = KafkaSubscriber.getTopicListSubscriber(topics); + return this; + } + + /** + * Set a list of topics the KafkaSource should consume from. All the topics in the list should + * have existed in the Kafka cluster. Otherwise an exception will be thrown. To allow some of + * the topics to be created lazily, please use {@link #setTopicPattern(Pattern)} instead. + * + * @param topics the list of topics to consume from. + * @return this KafkaSourceBuilder. + * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Collection) + */ + public KafkaSourceBuilder setTopics(String... topics) { + return setTopics(Arrays.asList(topics)); + } + + /** + * Set a topic pattern to consume from use the java {@link Pattern}. + * + * @param topicPattern the pattern of the topic name to consume from. + * @return this KafkaSourceBuilder. + * @see org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(Pattern) + */ + public KafkaSourceBuilder setTopicPattern(Pattern topicPattern) { + ensureSubscriberIsNull("topic pattern"); + subscriber = KafkaSubscriber.getTopicPatternSubscriber(topicPattern); + return this; + } + + /** + * Set a set of partitions to consume from. + * + * @param partitions the set of partitions to consume from. + * @return this KafkaSourceBuilder. + * @see org.apache.kafka.clients.consumer.KafkaConsumer#assign(Collection) + */ + public KafkaSourceBuilder setPartitions(Set partitions) { + ensureSubscriberIsNull("partitions"); + subscriber = KafkaSubscriber.getPartitionSetSubscriber(partitions); + return this; + } + + /** + * Set a custom Kafka subscriber to use to discover new splits. + * + * @param kafkaSubscriber the {@link KafkaSubscriber} to use for split discovery. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setKafkaSubscriber(KafkaSubscriber kafkaSubscriber) { + ensureSubscriberIsNull("custom"); + this.subscriber = checkNotNull(kafkaSubscriber); + return this; + } + + /** + * Specify from which offsets the KafkaSource should start consuming from by providing an {@link + * OffsetsInitializer}. + * + *

The following {@link OffsetsInitializer}s are commonly used and provided out of the box. + * Users can also implement their own {@link OffsetsInitializer} for custom behaviors. + * + *

    + *
  • {@link OffsetsInitializer#earliest()} - starting from the earliest offsets. This is + * also the default {@link OffsetsInitializer} of the KafkaSource for starting offsets. + *
  • {@link OffsetsInitializer#latest()} - starting from the latest offsets. + *
  • {@link OffsetsInitializer#committedOffsets()} - starting from the committed offsets of + * the consumer group. + *
  • {@link + * OffsetsInitializer#committedOffsets(org.apache.kafka.clients.consumer.OffsetResetStrategy)} + * - starting from the committed offsets of the consumer group. If there is no committed + * offsets, starting from the offsets specified by the {@link + * org.apache.kafka.clients.consumer.OffsetResetStrategy OffsetResetStrategy}. + *
  • {@link OffsetsInitializer#offsets(Map)} - starting from the specified offsets for each + * partition. + *
  • {@link OffsetsInitializer#timestamp(long)} - starting from the specified timestamp for + * each partition. Note that the guarantee here is that all the records in Kafka whose + * {@link org.apache.kafka.clients.consumer.ConsumerRecord#timestamp()} is greater than + * the given starting timestamp will be consumed. However, it is possible that some + * consumer records whose timestamp is smaller than the given starting timestamp are also + * consumed. + *
+ * + * @param startingOffsetsInitializer the {@link OffsetsInitializer} setting the starting offsets + * for the Source. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setStartingOffsets( + OffsetsInitializer startingOffsetsInitializer) { + this.startingOffsetsInitializer = startingOffsetsInitializer; + return this; + } + + /** + * By default the KafkaSource is set to run as {@link Boundedness#CONTINUOUS_UNBOUNDED} and thus + * never stops until the Flink job fails or is canceled. To let the KafkaSource run as a + * streaming source but still stop at some point, one can set an {@link OffsetsInitializer} to + * specify the stopping offsets for each partition. When all the partitions have reached their + * stopping offsets, the KafkaSource will then exit. + * + *

This method is different from {@link #setBounded(OffsetsInitializer)} in that after + * setting the stopping offsets with this method, {@link KafkaSource#getBoundedness()} will + * still return {@link Boundedness#CONTINUOUS_UNBOUNDED} even though it will stop at the + * stopping offsets specified by the stopping offsets {@link OffsetsInitializer}. + * + *

The following {@link OffsetsInitializer} are commonly used and provided out of the box. + * Users can also implement their own {@link OffsetsInitializer} for custom behaviors. + * + *

    + *
  • {@link OffsetsInitializer#latest()} - stop at the latest offsets of the partitions when + * the KafkaSource starts to run. + *
  • {@link OffsetsInitializer#committedOffsets()} - stops at the committed offsets of the + * consumer group. + *
  • {@link OffsetsInitializer#offsets(Map)} - stops at the specified offsets for each + * partition. + *
  • {@link OffsetsInitializer#timestamp(long)} - stops at the specified timestamp for each + * partition. The guarantee of setting the stopping timestamp is that no Kafka records + * whose {@link org.apache.kafka.clients.consumer.ConsumerRecord#timestamp()} is greater + * than the given stopping timestamp will be consumed. However, it is possible that some + * records whose timestamp is smaller than the specified stopping timestamp are not + * consumed. + *
+ * + * @param stoppingOffsetsInitializer The {@link OffsetsInitializer} to specify the stopping + * offset. + * @return this KafkaSourceBuilder. + * @see #setBounded(OffsetsInitializer) + */ + public KafkaSourceBuilder setUnbounded(OffsetsInitializer stoppingOffsetsInitializer) { + this.boundedness = Boundedness.CONTINUOUS_UNBOUNDED; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + return this; + } + + /** + * By default the KafkaSource is set to run as {@link Boundedness#CONTINUOUS_UNBOUNDED} and thus + * never stops until the Flink job fails or is canceled. To let the KafkaSource run as {@link + * Boundedness#BOUNDED} and stop at some point, one can set an {@link OffsetsInitializer} to + * specify the stopping offsets for each partition. When all the partitions have reached their + * stopping offsets, the KafkaSource will then exit. + * + *

This method is different from {@link #setUnbounded(OffsetsInitializer)} in that after + * setting the stopping offsets with this method, {@link KafkaSource#getBoundedness()} will + * return {@link Boundedness#BOUNDED} instead of {@link Boundedness#CONTINUOUS_UNBOUNDED}. + * + *

The following {@link OffsetsInitializer} are commonly used and provided out of the box. + * Users can also implement their own {@link OffsetsInitializer} for custom behaviors. + * + *

    + *
  • {@link OffsetsInitializer#latest()} - stop at the latest offsets of the partitions when + * the KafkaSource starts to run. + *
  • {@link OffsetsInitializer#committedOffsets()} - stops at the committed offsets of the + * consumer group. + *
  • {@link OffsetsInitializer#offsets(Map)} - stops at the specified offsets for each + * partition. + *
  • {@link OffsetsInitializer#timestamp(long)} - stops at the specified timestamp for each + * partition. The guarantee of setting the stopping timestamp is that no Kafka records + * whose {@link org.apache.kafka.clients.consumer.ConsumerRecord#timestamp()} is greater + * than the given stopping timestamp will be consumed. However, it is possible that some + * records whose timestamp is smaller than the specified stopping timestamp are not + * consumed. + *
+ * + * @param stoppingOffsetsInitializer the {@link OffsetsInitializer} to specify the stopping + * offsets. + * @return this KafkaSourceBuilder. + * @see #setUnbounded(OffsetsInitializer) + */ + public KafkaSourceBuilder setBounded(OffsetsInitializer stoppingOffsetsInitializer) { + this.boundedness = Boundedness.BOUNDED; + this.stoppingOffsetsInitializer = stoppingOffsetsInitializer; + return this; + } + + /** + * Sets the {@link KafkaRecordDeserializationSchema deserializer} of the {@link + * org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} for KafkaSource. + * + * @param recordDeserializer the deserializer for Kafka {@link + * org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord}. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setDeserializer( + KafkaRecordDeserializationSchema recordDeserializer) { + this.deserializationSchema = recordDeserializer; + return this; + } + + /** + * Sets the {@link KafkaRecordDeserializationSchema deserializer} of the {@link + * org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} for KafkaSource. The given + * {@link DeserializationSchema} will be used to deserialize the value of ConsumerRecord. The + * other information (e.g. key) in a ConsumerRecord will be ignored. + * + * @param deserializationSchema the {@link DeserializationSchema} to use for deserialization. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setValueOnlyDeserializer( + DeserializationSchema deserializationSchema) { + this.deserializationSchema = + KafkaRecordDeserializationSchema.valueOnly(deserializationSchema); + return this; + } + + /** + * Sets the client id prefix of this KafkaSource. + * + * @param prefix the client id prefix to use for this KafkaSource. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setClientIdPrefix(String prefix) { + return setProperty(KafkaSourceOptions.CLIENT_ID_PREFIX.key(), prefix); + } + + /** + * Set the clientRackId supplier to be passed down to the KafkaPartitionSplitReader. + * + * @param rackIdCallback callback to provide Kafka consumer client.rack + * @return this KafkaSourceBuilder + */ + public KafkaSourceBuilder setRackIdSupplier(SerializableSupplier rackIdCallback) { + this.rackIdSupplier = rackIdCallback; + return this; + } + + /** + * Set an arbitrary property for the KafkaSource and KafkaConsumer. The valid keys can be found + * in {@link ConsumerConfig} and {@link KafkaSourceOptions}. + * + *

Note that the following keys will be overridden by the builder when the KafkaSource is + * created. + * + *

    + *
  • key.deserializer is always set to {@link ByteArrayDeserializer}. + *
  • value.deserializer is always set to {@link ByteArrayDeserializer}. + *
  • auto.offset.reset.strategy is overridden by {@link + * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by + * default {@link OffsetsInitializer#earliest()}. + *
  • partition.discovery.interval.ms is overridden to -1 when {@link + * #setBounded(OffsetsInitializer)} has been invoked. + *
+ * + * @param key the key of the property. + * @param value the value of the property. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setProperty(String key, String value) { + props.setProperty(key, value); + return this; + } + + /** + * Set arbitrary properties for the KafkaSource and KafkaConsumer. The valid keys can be found + * in {@link ConsumerConfig} and {@link KafkaSourceOptions}. + * + *

Note that the following keys will be overridden by the builder when the KafkaSource is + * created. + * + *

    + *
  • key.deserializer is always set to {@link ByteArrayDeserializer}. + *
  • value.deserializer is always set to {@link ByteArrayDeserializer}. + *
  • auto.offset.reset.strategy is overridden by {@link + * OffsetsInitializer#getAutoOffsetResetStrategy()} for the starting offsets, which is by + * default {@link OffsetsInitializer#earliest()}. + *
  • partition.discovery.interval.ms is overridden to -1 when {@link + * #setBounded(OffsetsInitializer)} has been invoked. + *
  • client.id is overridden to the "client.id.prefix-RANDOM_LONG", or + * "group.id-RANDOM_LONG" if the client id prefix is not set. + *
+ * + * @param props the properties to set for the KafkaSource. + * @return this KafkaSourceBuilder. + */ + public KafkaSourceBuilder setProperties(Properties props) { + this.props.putAll(props); + return this; + } + + /** + * Build the {@link KafkaSource}. + * + * @return a KafkaSource with the settings made for this builder. + */ + public KafkaSource build() { + sanityCheck(); + parseAndSetRequiredProperties(); + return new KafkaSource<>( + subscriber, + startingOffsetsInitializer, + stoppingOffsetsInitializer, + boundedness, + deserializationSchema, + props, + rackIdSupplier); + } + + // ------------- private helpers -------------- + + private void ensureSubscriberIsNull(String attemptingSubscribeMode) { + if (subscriber != null) { + throw new IllegalStateException( + String.format( + "Cannot use %s for consumption because a %s is already set for consumption.", + attemptingSubscribeMode, subscriber.getClass().getSimpleName())); + } + } + + private void parseAndSetRequiredProperties() { + maybeOverride( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + true); + maybeOverride( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + ByteArrayDeserializer.class.getName(), + true); + if (!props.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + LOG.warn( + "Offset commit on checkpoint is disabled because {} is not specified", + ConsumerConfig.GROUP_ID_CONFIG); + maybeOverride(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key(), "false", false); + } + maybeOverride(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false", false); + maybeOverride( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + startingOffsetsInitializer.getAutoOffsetResetStrategy().name().toLowerCase(), + true); + + // If the source is bounded, do not run periodic partition discovery. + maybeOverride( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), + "-1", + boundedness == Boundedness.BOUNDED); + + // If the client id prefix is not set, reuse the consumer group id as the client id prefix, + // or generate a random string if consumer group id is not specified. + maybeOverride( + KafkaSourceOptions.CLIENT_ID_PREFIX.key(), + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) + ? props.getProperty(ConsumerConfig.GROUP_ID_CONFIG) + : "KafkaSource-" + new Random().nextLong(), + false); + } + + private boolean maybeOverride(String key, String value, boolean override) { + boolean overridden = false; + String userValue = props.getProperty(key); + if (userValue != null) { + if (override) { + LOG.warn( + String.format( + "Property %s is provided but will be overridden from %s to %s", + key, userValue, value)); + props.setProperty(key, value); + overridden = true; + } + } else { + props.setProperty(key, value); + } + return overridden; + } + + private void sanityCheck() { + // Check required configs. + for (String requiredConfig : REQUIRED_CONFIGS) { + checkNotNull( + props.getProperty(requiredConfig), + String.format("Property %s is required but not provided", requiredConfig)); + } + // Check required settings. + checkNotNull( + subscriber, + "No subscribe mode is specified, " + + "should be one of topics, topic pattern and partition set."); + checkNotNull(deserializationSchema, "Deserialization schema is required but not provided."); + // Check consumer group ID + checkState( + props.containsKey(ConsumerConfig.GROUP_ID_CONFIG) || !offsetCommitEnabledManually(), + String.format( + "Property %s is required when offset commit is enabled", + ConsumerConfig.GROUP_ID_CONFIG)); + // Check offsets initializers + if (startingOffsetsInitializer instanceof OffsetsInitializerValidator) { + ((OffsetsInitializerValidator) startingOffsetsInitializer).validate(props); + } + if (stoppingOffsetsInitializer instanceof OffsetsInitializerValidator) { + ((OffsetsInitializerValidator) stoppingOffsetsInitializer).validate(props); + } + } + + private boolean offsetCommitEnabledManually() { + boolean autoCommit = + props.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG) + && Boolean.parseBoolean( + props.getProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)); + boolean commitOnCheckpoint = + props.containsKey(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key()) + && Boolean.parseBoolean( + props.getProperty( + KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key())); + return autoCommit || commitOnCheckpoint; + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/reader/KafkaSourceReader.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/reader/KafkaSourceReader.java new file mode 100644 index 00000000000..0af906bea7b --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/reader/KafkaSourceReader.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.source.reader; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.connector.kafka.source.metrics.KafkaSourceReaderMetrics; +import org.apache.flink.connector.kafka.source.reader.fetcher.KafkaSourceFetcherManager; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplit; +import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitState; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** The source reader for Kafka partitions. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +// TODO: Add some method to make report audit information exactly once +public class KafkaSourceReader + extends + SingleThreadMultiplexSourceReaderBase, T, KafkaPartitionSplit, KafkaPartitionSplitState> { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceReader.class); + // These maps need to be concurrent because it will be accessed by both the main thread + // and the split fetcher thread in the callback. + private final SortedMap> offsetsToCommit; + private final ConcurrentMap offsetsOfFinishedSplits; + private final KafkaSourceReaderMetrics kafkaSourceReaderMetrics; + private final boolean commitOffsetsOnCheckpoint; + + public KafkaSourceReader( + FutureCompletingBlockingQueue>> elementsQueue, + KafkaSourceFetcherManager kafkaSourceFetcherManager, + RecordEmitter, T, KafkaPartitionSplitState> recordEmitter, + Configuration config, + SourceReaderContext context, + KafkaSourceReaderMetrics kafkaSourceReaderMetrics) { + super(elementsQueue, kafkaSourceFetcherManager, recordEmitter, config, context); + this.offsetsToCommit = Collections.synchronizedSortedMap(new TreeMap<>()); + this.offsetsOfFinishedSplits = new ConcurrentHashMap<>(); + this.kafkaSourceReaderMetrics = kafkaSourceReaderMetrics; + this.commitOffsetsOnCheckpoint = + config.get(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT); + if (!commitOffsetsOnCheckpoint) { + LOG.warn( + "Offset commit on checkpoint is disabled. " + + "Consuming offset will not be reported back to Kafka cluster."); + } + } + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + finishedSplitIds.forEach( + (ignored, splitState) -> { + if (splitState.getCurrentOffset() >= 0) { + offsetsOfFinishedSplits.put( + splitState.getTopicPartition(), + new OffsetAndMetadata(splitState.getCurrentOffset())); + } + }); + } + + @Override + public List snapshotState(long checkpointId) { + List splits = super.snapshotState(checkpointId); + if (!commitOffsetsOnCheckpoint) { + return splits; + } + + if (splits.isEmpty() && offsetsOfFinishedSplits.isEmpty()) { + offsetsToCommit.put(checkpointId, Collections.emptyMap()); + } else { + Map offsetsMap = + offsetsToCommit.computeIfAbsent(checkpointId, id -> new HashMap<>()); + // Put the offsets of the active splits. + for (KafkaPartitionSplit split : splits) { + // If the checkpoint is triggered before the partition starting offsets + // is retrieved, do not commit the offsets for those partitions. + if (split.getStartingOffset() >= 0) { + offsetsMap.put( + split.getTopicPartition(), + new OffsetAndMetadata(split.getStartingOffset())); + } + } + // Put offsets of all the finished splits. + offsetsMap.putAll(offsetsOfFinishedSplits); + } + return splits; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + LOG.debug("Committing offsets for checkpoint {}", checkpointId); + if (!commitOffsetsOnCheckpoint) { + return; + } + + Map committedPartitions = + offsetsToCommit.get(checkpointId); + if (committedPartitions == null) { + LOG.debug("Offsets for checkpoint {} have already been committed.", checkpointId); + return; + } + + if (committedPartitions.isEmpty()) { + LOG.debug("There are no offsets to commit for checkpoint {}.", checkpointId); + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); + return; + } + + ((KafkaSourceFetcherManager) splitFetcherManager) + .commitOffsets( + committedPartitions, + (ignored, e) -> { + // The offset commit here is needed by the external monitoring. It won't + // break Flink job's correctness if we fail to commit the offset here. + if (e != null) { + kafkaSourceReaderMetrics.recordFailedCommit(); + LOG.warn( + "Failed to commit consumer offsets for checkpoint {}", + checkpointId, + e); + } else { + LOG.debug( + "Successfully committed offsets for checkpoint {}", + checkpointId); + kafkaSourceReaderMetrics.recordSucceededCommit(); + // If the finished topic partition has been committed, we remove it + // from the offsets of the finished splits map. + committedPartitions.forEach( + (tp, offset) -> kafkaSourceReaderMetrics.recordCommittedOffset( + tp, offset.offset())); + offsetsOfFinishedSplits + .entrySet() + .removeIf( + entry -> committedPartitions.containsKey( + entry.getKey())); + removeAllOffsetsToCommitUpToCheckpoint(checkpointId); + } + }); + } + + private void removeAllOffsetsToCommitUpToCheckpoint(long checkpointId) { + while (!offsetsToCommit.isEmpty() && offsetsToCommit.firstKey() <= checkpointId) { + offsetsToCommit.remove(offsetsToCommit.firstKey()); + } + } + + @Override + protected KafkaPartitionSplitState initializedState(KafkaPartitionSplit split) { + return new KafkaPartitionSplitState(split); + } + + @Override + protected KafkaPartitionSplit toSplitType(String splitId, KafkaPartitionSplitState splitState) { + return splitState.toKafkaPartitionSplit(); + } + + // ------------------------ + + @VisibleForTesting + SortedMap> getOffsetsToCommit() { + return offsetsToCommit; + } + + @VisibleForTesting + int getNumAliveFetchers() { + return splitFetcherManager.getNumAliveFetchers(); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaDeserializationSchema.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaDeserializationSchema.java new file mode 100644 index 00000000000..2f35af5e23c --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaDeserializationSchema.java @@ -0,0 +1,277 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import org.apache.flink.streaming.connectors.kafka.KafkaSerializationSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.DeserializationException; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; +import org.apache.kafka.clients.consumer.ConsumerRecord; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +/** A specific {@link KafkaSerializationSchema} for {@link KafkaDynamicSource}. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +// TODO: support SourceExactlyMetric and add metric collection points +class DynamicKafkaDeserializationSchema implements KafkaDeserializationSchema { + + private static final long serialVersionUID = 1L; + + private final @Nullable DeserializationSchema keyDeserialization; + + private final DeserializationSchema valueDeserialization; + + private final boolean hasMetadata; + + private final BufferingCollector keyCollector; + + private final OutputProjectionCollector outputCollector; + + private final TypeInformation producedTypeInfo; + + private final boolean upsertMode; + + DynamicKafkaDeserializationSchema( + int physicalArity, + @Nullable DeserializationSchema keyDeserialization, + int[] keyProjection, + DeserializationSchema valueDeserialization, + int[] valueProjection, + boolean hasMetadata, + MetadataConverter[] metadataConverters, + TypeInformation producedTypeInfo, + boolean upsertMode) { + if (upsertMode) { + Preconditions.checkArgument( + keyDeserialization != null && keyProjection.length > 0, + "Key must be set in upsert mode for deserialization schema."); + } + this.keyDeserialization = keyDeserialization; + this.valueDeserialization = valueDeserialization; + this.hasMetadata = hasMetadata; + this.keyCollector = new BufferingCollector(); + this.outputCollector = + new OutputProjectionCollector( + physicalArity, + keyProjection, + valueProjection, + metadataConverters, + upsertMode); + this.producedTypeInfo = producedTypeInfo; + this.upsertMode = upsertMode; + } + + @Override + public void open(DeserializationSchema.InitializationContext context) throws Exception { + if (keyDeserialization != null) { + keyDeserialization.open(context); + } + valueDeserialization.open(context); + } + + @Override + public boolean isEndOfStream(RowData nextElement) { + return false; + } + + @Override + public RowData deserialize(ConsumerRecord record) throws Exception { + throw new IllegalStateException("A collector is required for deserializing."); + } + + @Override + public void deserialize(ConsumerRecord record, Collector collector) + throws Exception { + // shortcut in case no output projection is required, + // also not for a cartesian product with the keys + if (keyDeserialization == null && !hasMetadata) { + valueDeserialization.deserialize(record.value(), collector); + return; + } + + // buffer key(s) + if (keyDeserialization != null) { + keyDeserialization.deserialize(record.key(), keyCollector); + } + + // project output while emitting values + outputCollector.inputRecord = record; + outputCollector.physicalKeyRows = keyCollector.buffer; + outputCollector.outputCollector = collector; + if (record.value() == null && upsertMode) { + // collect tombstone messages in upsert mode by hand + outputCollector.collect(null); + } else { + valueDeserialization.deserialize(record.value(), outputCollector); + } + keyCollector.buffer.clear(); + } + + @Override + public TypeInformation getProducedType() { + return producedTypeInfo; + } + + // -------------------------------------------------------------------------------------------- + + interface MetadataConverter extends Serializable { + + Object read(ConsumerRecord record); + } + + // -------------------------------------------------------------------------------------------- + + private static final class BufferingCollector implements Collector, Serializable { + + private static final long serialVersionUID = 1L; + + private final List buffer = new ArrayList<>(); + + @Override + public void collect(RowData record) { + buffer.add(record); + } + + @Override + public void close() { + // nothing to do + } + } + + // -------------------------------------------------------------------------------------------- + + /** + * Emits a row with key, value, and metadata fields. + * + *

The collector is able to handle the following kinds of keys: + * + *

    + *
  • No key is used. + *
  • A key is used. + *
  • The deserialization schema emits multiple keys. + *
  • Keys and values have overlapping fields. + *
  • Keys are used and value is null. + *
+ */ + private static final class OutputProjectionCollector + implements + Collector, + Serializable { + + private static final long serialVersionUID = 1L; + + private final int physicalArity; + + private final int[] keyProjection; + + private final int[] valueProjection; + + private final MetadataConverter[] metadataConverters; + + private final boolean upsertMode; + + private transient ConsumerRecord inputRecord; + + private transient List physicalKeyRows; + + private transient Collector outputCollector; + + OutputProjectionCollector( + int physicalArity, + int[] keyProjection, + int[] valueProjection, + MetadataConverter[] metadataConverters, + boolean upsertMode) { + this.physicalArity = physicalArity; + this.keyProjection = keyProjection; + this.valueProjection = valueProjection; + this.metadataConverters = metadataConverters; + this.upsertMode = upsertMode; + } + + @Override + public void collect(RowData physicalValueRow) { + // no key defined + if (keyProjection.length == 0) { + emitRow(null, (GenericRowData) physicalValueRow); + return; + } + + // otherwise emit a value for each key + for (RowData physicalKeyRow : physicalKeyRows) { + emitRow((GenericRowData) physicalKeyRow, (GenericRowData) physicalValueRow); + } + } + + @Override + public void close() { + // nothing to do + } + + private void emitRow( + @Nullable GenericRowData physicalKeyRow, + @Nullable GenericRowData physicalValueRow) { + final RowKind rowKind; + if (physicalValueRow == null) { + if (upsertMode) { + rowKind = RowKind.DELETE; + } else { + throw new DeserializationException( + "Invalid null value received in non-upsert mode. Could not to set row kind for output record."); + } + } else { + rowKind = physicalValueRow.getRowKind(); + } + + final int metadataArity = metadataConverters.length; + final GenericRowData producedRow = + new GenericRowData(rowKind, physicalArity + metadataArity); + + for (int keyPos = 0; keyPos < keyProjection.length; keyPos++) { + assert physicalKeyRow != null; + producedRow.setField(keyProjection[keyPos], physicalKeyRow.getField(keyPos)); + } + + if (physicalValueRow != null) { + for (int valuePos = 0; valuePos < valueProjection.length; valuePos++) { + producedRow.setField( + valueProjection[valuePos], physicalValueRow.getField(valuePos)); + } + } + + for (int metadataPos = 0; metadataPos < metadataArity; metadataPos++) { + producedRow.setField( + physicalArity + metadataPos, + metadataConverters[metadataPos].read(inputRecord)); + } + + outputCollector.collect(producedRow); + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaRecordSerializationSchema.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaRecordSerializationSchema.java new file mode 100644 index 00000000000..7ff52be24d7 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaRecordSerializationSchema.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema; +import org.apache.flink.connector.kafka.sink.KafkaSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Preconditions; +import org.apache.kafka.clients.producer.ProducerRecord; + +import javax.annotation.Nullable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** SerializationSchema used by {@link KafkaDynamicSink} to configure a {@link KafkaSink}. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +class DynamicKafkaRecordSerializationSchema implements KafkaRecordSerializationSchema { + + private final String topic; + private final FlinkKafkaPartitioner partitioner; + @Nullable + private final SerializationSchema keySerialization; + private final SerializationSchema valueSerialization; + private final RowData.FieldGetter[] keyFieldGetters; + private final RowData.FieldGetter[] valueFieldGetters; + private final boolean hasMetadata; + private final int[] metadataPositions; + private final boolean upsertMode; + + DynamicKafkaRecordSerializationSchema( + String topic, + @Nullable FlinkKafkaPartitioner partitioner, + @Nullable SerializationSchema keySerialization, + SerializationSchema valueSerialization, + RowData.FieldGetter[] keyFieldGetters, + RowData.FieldGetter[] valueFieldGetters, + boolean hasMetadata, + int[] metadataPositions, + boolean upsertMode) { + if (upsertMode) { + Preconditions.checkArgument( + keySerialization != null && keyFieldGetters.length > 0, + "Key must be set in upsert mode for serialization schema."); + } + this.topic = checkNotNull(topic); + this.partitioner = partitioner; + this.keySerialization = keySerialization; + this.valueSerialization = checkNotNull(valueSerialization); + this.keyFieldGetters = keyFieldGetters; + this.valueFieldGetters = valueFieldGetters; + this.hasMetadata = hasMetadata; + this.metadataPositions = metadataPositions; + this.upsertMode = upsertMode; + } + + @Override + public ProducerRecord serialize( + RowData consumedRow, KafkaSinkContext context, Long timestamp) { + // shortcut in case no input projection is required + if (keySerialization == null && !hasMetadata) { + final byte[] valueSerialized = valueSerialization.serialize(consumedRow); + return new ProducerRecord<>( + topic, + extractPartition( + consumedRow, + null, + valueSerialized, + context.getPartitionsForTopic(topic)), + null, + valueSerialized); + } + final byte[] keySerialized; + if (keySerialization == null) { + keySerialized = null; + } else { + final RowData keyRow = createProjectedRow(consumedRow, RowKind.INSERT, keyFieldGetters); + keySerialized = keySerialization.serialize(keyRow); + } + + final byte[] valueSerialized; + final RowKind kind = consumedRow.getRowKind(); + if (upsertMode) { + if (kind == RowKind.DELETE || kind == RowKind.UPDATE_BEFORE) { + // transform the message as the tombstone message + valueSerialized = null; + } else { + // make the message to be INSERT to be compliant with the INSERT-ONLY format + final RowData valueRow = + DynamicKafkaRecordSerializationSchema.createProjectedRow( + consumedRow, kind, valueFieldGetters); + valueRow.setRowKind(RowKind.INSERT); + valueSerialized = valueSerialization.serialize(valueRow); + } + } else { + final RowData valueRow = + DynamicKafkaRecordSerializationSchema.createProjectedRow( + consumedRow, kind, valueFieldGetters); + valueSerialized = valueSerialization.serialize(valueRow); + } + + return new ProducerRecord<>( + topic, + extractPartition( + consumedRow, + keySerialized, + valueSerialized, + context.getPartitionsForTopic(topic)), + readMetadata(consumedRow, KafkaDynamicSink.WritableMetadata.TIMESTAMP), + keySerialized, + valueSerialized, + readMetadata(consumedRow, KafkaDynamicSink.WritableMetadata.HEADERS)); + } + + @Override + public void open( + SerializationSchema.InitializationContext context, KafkaSinkContext sinkContext) + throws Exception { + if (keySerialization != null) { + keySerialization.open(context); + } + if (partitioner != null) { + partitioner.open( + sinkContext.getParallelInstanceId(), + sinkContext.getNumberOfParallelInstances()); + } + valueSerialization.open(context); + } + + private Integer extractPartition( + RowData consumedRow, + @Nullable byte[] keySerialized, + byte[] valueSerialized, + int[] partitions) { + if (partitioner != null) { + return partitioner.partition( + consumedRow, keySerialized, valueSerialized, topic, partitions); + } + return null; + } + + static RowData createProjectedRow( + RowData consumedRow, RowKind kind, RowData.FieldGetter[] fieldGetters) { + final int arity = fieldGetters.length; + final GenericRowData genericRowData = new GenericRowData(kind, arity); + for (int fieldPos = 0; fieldPos < arity; fieldPos++) { + genericRowData.setField(fieldPos, fieldGetters[fieldPos].getFieldOrNull(consumedRow)); + } + return genericRowData; + } + + @SuppressWarnings("unchecked") + private T readMetadata(RowData consumedRow, KafkaDynamicSink.WritableMetadata metadata) { + final int pos = metadataPositions[metadata.ordinal()]; + if (pos < 0) { + return null; + } + return (T) metadata.converter.read(consumedRow, pos); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptions.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptions.java new file mode 100644 index 00000000000..98d1f62abd9 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptions.java @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.DescribedEnum; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.configuration.description.InlineElement; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.table.factories.FactoryUtil; + +import java.time.Duration; +import java.util.List; + +import static org.apache.flink.configuration.description.TextElement.text; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT_SUFFIX; + +/** + * Options for the Kafka connector. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +public class KafkaConnectorOptions { + + // -------------------------------------------------------------------------------------------- + // Format options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption KEY_FORMAT = + ConfigOptions.key("key" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for encoding key data. " + + "The identifier is used to discover a suitable format factory."); + + public static final ConfigOption VALUE_FORMAT = + ConfigOptions.key("value" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for encoding value data. " + + "The identifier is used to discover a suitable format factory."); + + public static final ConfigOption> KEY_FIELDS = + ConfigOptions.key("key.fields") + .stringType() + .asList() + .defaultValues() + .withDescription( + "Defines an explicit list of physical columns from the table schema " + + "that configure the data type for the key format. By default, this list is " + + "empty and thus a key is undefined."); + + public static final ConfigOption VALUE_FIELDS_INCLUDE = + ConfigOptions.key("value.fields-include") + .enumType(ValueFieldsStrategy.class) + .defaultValue(ValueFieldsStrategy.ALL) + .withDescription( + String.format( + "Defines a strategy how to deal with key columns in the data type " + + "of the value format. By default, '%s' physical columns of the table schema " + + "will be included in the value format which means that the key columns " + + "appear in the data type for both the key and value format.", + ValueFieldsStrategy.ALL)); + + public static final ConfigOption KEY_FIELDS_PREFIX = + ConfigOptions.key("key.fields-prefix") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text( + "Defines a custom prefix for all fields of the key format to avoid " + + "name clashes with fields of the value format. " + + "By default, the prefix is empty.") + .linebreak() + .text( + String.format( + "If a custom prefix is defined, both the table schema and '%s' will work with prefixed names.", + KEY_FIELDS.key())) + .linebreak() + .text( + "When constructing the data type of the key format, the prefix " + + "will be removed and the non-prefixed names will be used within the key format.") + .linebreak() + .text( + String.format( + "Please note that this option requires that '%s' must be '%s'.", + VALUE_FIELDS_INCLUDE.key(), + ValueFieldsStrategy.EXCEPT_KEY)) + .build()); + + public static final ConfigOption SINK_PARALLELISM = FactoryUtil.SINK_PARALLELISM; + + // -------------------------------------------------------------------------------------------- + // Kafka specific options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption> TOPIC = + ConfigOptions.key("topic") + .stringType() + .asList() + .noDefaultValue() + .withDescription( + "Topic names from which the table is read. Either 'topic' or 'topic-pattern' must be set for source. " + + "Option 'topic' is required for sink."); + + public static final ConfigOption TOPIC_PATTERN = + ConfigOptions.key("topic-pattern") + .stringType() + .noDefaultValue() + .withDescription( + "Optional topic pattern from which the table is read for source. Either 'topic' or 'topic-pattern' must be set."); + + public static final ConfigOption PROPS_BOOTSTRAP_SERVERS = + ConfigOptions.key("properties.bootstrap.servers") + .stringType() + .noDefaultValue() + .withDescription("Required Kafka server connection string"); + + public static final ConfigOption PROPS_GROUP_ID = + ConfigOptions.key("properties.group.id") + .stringType() + .noDefaultValue() + .withDescription( + "Required consumer group in Kafka consumer, no need for Kafka producer"); + + // -------------------------------------------------------------------------------------------- + // Scan specific options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SCAN_STARTUP_MODE = + ConfigOptions.key("scan.startup.mode") + .enumType(ScanStartupMode.class) + .defaultValue(ScanStartupMode.GROUP_OFFSETS) + .withDescription("Startup mode for Kafka consumer."); + + public static final ConfigOption SCAN_BOUNDED_MODE = + ConfigOptions.key("scan.bounded.mode") + .enumType(ScanBoundedMode.class) + .defaultValue(ScanBoundedMode.UNBOUNDED) + .withDescription("Bounded mode for Kafka consumer."); + + public static final ConfigOption SCAN_STARTUP_SPECIFIC_OFFSETS = + ConfigOptions.key("scan.startup.specific-offsets") + .stringType() + .noDefaultValue() + .withDescription( + "Optional offsets used in case of \"specific-offsets\" startup mode"); + + public static final ConfigOption SCAN_BOUNDED_SPECIFIC_OFFSETS = + ConfigOptions.key("scan.bounded.specific-offsets") + .stringType() + .noDefaultValue() + .withDescription( + "Optional offsets used in case of \"specific-offsets\" bounded mode"); + + public static final ConfigOption SCAN_STARTUP_TIMESTAMP_MILLIS = + ConfigOptions.key("scan.startup.timestamp-millis") + .longType() + .noDefaultValue() + .withDescription( + "Optional timestamp used in case of \"timestamp\" startup mode"); + + public static final ConfigOption SCAN_BOUNDED_TIMESTAMP_MILLIS = + ConfigOptions.key("scan.bounded.timestamp-millis") + .longType() + .noDefaultValue() + .withDescription( + "Optional timestamp used in case of \"timestamp\" bounded mode"); + + public static final ConfigOption SCAN_TOPIC_PARTITION_DISCOVERY = + ConfigOptions.key("scan.topic-partition-discovery.interval") + .durationType() + .defaultValue(Duration.ofMinutes(5)) + .withDescription( + "Optional interval for consumer to discover dynamically created Kafka partitions periodically." + + "The value 0 disables the partition discovery." + + "The default value is 5 minutes, which is equal to the default value of metadata.max.age.ms in Kafka."); + + // -------------------------------------------------------------------------------------------- + // Sink specific options + // -------------------------------------------------------------------------------------------- + + public static final ConfigOption SINK_PARTITIONER = + ConfigOptions.key("sink.partitioner") + .stringType() + .defaultValue("default") + .withDescription( + Description.builder() + .text( + "Optional output partitioning from Flink's partitions into Kafka's partitions. Valid enumerations are") + .list( + text( + "'default' (use kafka default partitioner to partition records)"), + text( + "'fixed' (each Flink partition ends up in at most one Kafka partition)"), + text( + "'round-robin' (a Flink partition is distributed to Kafka partitions round-robin when 'key.fields' is not specified)"), + text( + "custom class name (use custom FlinkKafkaPartitioner subclass)")) + .build()); + + // Disable this feature by default + public static final ConfigOption SINK_BUFFER_FLUSH_MAX_ROWS = + ConfigOptions.key("sink.buffer-flush.max-rows") + .intType() + .defaultValue(0) + .withDescription( + Description.builder() + .text( + "The max size of buffered records before flushing. " + + "When the sink receives many updates on the same key, " + + "the buffer will retain the last records of the same key. " + + "This can help to reduce data shuffling and avoid possible tombstone messages to the Kafka topic.") + .linebreak() + .text("Can be set to '0' to disable it.") + .linebreak() + .text( + "Note both 'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' " + + "must be set to be greater than zero to enable sink buffer flushing.") + .build()); + + // Disable this feature by default + public static final ConfigOption SINK_BUFFER_FLUSH_INTERVAL = + ConfigOptions.key("sink.buffer-flush.interval") + .durationType() + .defaultValue(Duration.ofSeconds(0)) + .withDescription( + Description.builder() + .text( + "The flush interval millis. Over this time, asynchronous threads " + + "will flush data. When the sink receives many updates on the same key, " + + "the buffer will retain the last record of the same key.") + .linebreak() + .text("Can be set to '0' to disable it.") + .linebreak() + .text( + "Note both 'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' " + + "must be set to be greater than zero to enable sink buffer flushing.") + .build()); + + public static final ConfigOption DELIVERY_GUARANTEE = + ConfigOptions.key("sink.delivery-guarantee") + .enumType(DeliveryGuarantee.class) + .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE) + .withDescription("Optional delivery guarantee when committing."); + + public static final ConfigOption TRANSACTIONAL_ID_PREFIX = + ConfigOptions.key("sink.transactional-id-prefix") + .stringType() + .noDefaultValue() + .withDescription( + "If the delivery guarantee is configured as " + + DeliveryGuarantee.EXACTLY_ONCE + + " this value is used a prefix for the identifier of all opened Kafka transactions."); + + // -------------------------------------------------------------------------------------------- + // Enums + // -------------------------------------------------------------------------------------------- + + /** + * Strategies to derive the data type of a value format by considering a key format. + */ + public enum ValueFieldsStrategy { + ALL, + EXCEPT_KEY + } + + /** + * Startup mode for the Kafka consumer, see {@link #SCAN_STARTUP_MODE}. + */ + public enum ScanStartupMode implements DescribedEnum { + + EARLIEST_OFFSET("earliest-offset", text("Start from the earliest offset possible.")), + LATEST_OFFSET("latest-offset", text("Start from the latest offset.")), + GROUP_OFFSETS( + "group-offsets", + text( + "Start from committed offsets in ZooKeeper / Kafka brokers of a specific consumer group.")), + TIMESTAMP("timestamp", text("Start from user-supplied timestamp for each partition.")), + SPECIFIC_OFFSETS( + "specific-offsets", + text("Start from user-supplied specific offsets for each partition.")); + + private final String value; + private final InlineElement description; + + ScanStartupMode(String value, InlineElement description) { + this.value = value; + this.description = description; + } + + @Override + public String toString() { + return value; + } + + @Override + public InlineElement getDescription() { + return description; + } + } + + /** + * Bounded mode for the Kafka consumer, see {@link #SCAN_BOUNDED_MODE}. + */ + public enum ScanBoundedMode implements DescribedEnum { + + UNBOUNDED("unbounded", text("Do not stop consuming")), + LATEST_OFFSET( + "latest-offset", + text( + "Bounded by latest offsets. This is evaluated at the start of consumption" + + " from a given partition.")), + GROUP_OFFSETS( + "group-offsets", + text( + "Bounded by committed offsets in ZooKeeper / Kafka brokers of a specific" + + " consumer group. This is evaluated at the start of consumption" + + " from a given partition.")), + TIMESTAMP("timestamp", text("Bounded by a user-supplied timestamp.")), + SPECIFIC_OFFSETS( + "specific-offsets", + text( + "Bounded by user-supplied specific offsets for each partition. If an offset" + + " for a partition is not provided it will not consume from that" + + " partition.")); + private final String value; + private final InlineElement description; + + ScanBoundedMode(String value, InlineElement description) { + this.value = value; + this.description = description; + } + + @Override + public String toString() { + return value; + } + + @Override + public InlineElement getDescription() { + return description; + } + } + + private KafkaConnectorOptions() { + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptionsUtil.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptionsUtil.java new file mode 100644 index 00000000000..63d6b6250a1 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptionsUtil.java @@ -0,0 +1,699 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; +import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanStartupMode; +import org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ValueFieldsStrategy; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.regex.Pattern; +import java.util.stream.IntStream; + +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARTITIONER; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC_PATTERN; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT; + +/** Utilities for {@link KafkaConnectorOptions}. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +class KafkaConnectorOptionsUtil { + + private static final ConfigOption SCHEMA_REGISTRY_SUBJECT = + ConfigOptions.key("schema-registry.subject").stringType().noDefaultValue(); + + // -------------------------------------------------------------------------------------------- + // Option enumerations + // -------------------------------------------------------------------------------------------- + + // Sink partitioner. + public static final String SINK_PARTITIONER_VALUE_DEFAULT = "default"; + public static final String SINK_PARTITIONER_VALUE_FIXED = "fixed"; + public static final String SINK_PARTITIONER_VALUE_ROUND_ROBIN = "round-robin"; + + // Prefix for Kafka specific properties. + public static final String PROPERTIES_PREFIX = "properties."; + + // Other keywords. + private static final String PARTITION = "partition"; + private static final String OFFSET = "offset"; + protected static final String AVRO_CONFLUENT = "avro-confluent"; + protected static final String DEBEZIUM_AVRO_CONFLUENT = "debezium-avro-confluent"; + private static final List SCHEMA_REGISTRY_FORMATS = + Arrays.asList(AVRO_CONFLUENT, DEBEZIUM_AVRO_CONFLUENT); + + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- + + public static void validateTableSourceOptions(ReadableConfig tableOptions) { + validateSourceTopic(tableOptions); + validateScanStartupMode(tableOptions); + validateScanBoundedMode(tableOptions); + } + + public static void validateTableSinkOptions(ReadableConfig tableOptions) { + validateSinkTopic(tableOptions); + validateSinkPartitioner(tableOptions); + } + + public static void validateSourceTopic(ReadableConfig tableOptions) { + Optional> topic = tableOptions.getOptional(TOPIC); + Optional pattern = tableOptions.getOptional(TOPIC_PATTERN); + + if (topic.isPresent() && pattern.isPresent()) { + throw new ValidationException( + "Option 'topic' and 'topic-pattern' shouldn't be set together."); + } + + if (!topic.isPresent() && !pattern.isPresent()) { + throw new ValidationException("Either 'topic' or 'topic-pattern' must be set."); + } + } + + public static void validateSinkTopic(ReadableConfig tableOptions) { + String errorMessageTemp = + "Flink Kafka sink currently only supports single topic, but got %s: %s."; + if (!isSingleTopic(tableOptions)) { + if (tableOptions.getOptional(TOPIC_PATTERN).isPresent()) { + throw new ValidationException( + String.format( + errorMessageTemp, + "'topic-pattern'", + tableOptions.get(TOPIC_PATTERN))); + } else { + throw new ValidationException( + String.format(errorMessageTemp, "'topic'", tableOptions.get(TOPIC))); + } + } + } + + private static void validateScanStartupMode(ReadableConfig tableOptions) { + tableOptions + .getOptional(SCAN_STARTUP_MODE) + .ifPresent( + mode -> { + switch (mode) { + case TIMESTAMP: + if (!tableOptions + .getOptional(SCAN_STARTUP_TIMESTAMP_MILLIS) + .isPresent()) { + throw new ValidationException( + String.format( + "'%s' is required in '%s' startup mode" + + " but missing.", + SCAN_STARTUP_TIMESTAMP_MILLIS.key(), + ScanStartupMode.TIMESTAMP)); + } + + break; + case SPECIFIC_OFFSETS: + if (!tableOptions + .getOptional(SCAN_STARTUP_SPECIFIC_OFFSETS) + .isPresent()) { + throw new ValidationException( + String.format( + "'%s' is required in '%s' startup mode" + + " but missing.", + SCAN_STARTUP_SPECIFIC_OFFSETS.key(), + ScanStartupMode.SPECIFIC_OFFSETS)); + } + if (!isSingleTopic(tableOptions)) { + throw new ValidationException( + "Currently Kafka source only supports specific offset for single topic."); + } + String specificOffsets = + tableOptions.get(SCAN_STARTUP_SPECIFIC_OFFSETS); + parseSpecificOffsets( + specificOffsets, SCAN_STARTUP_SPECIFIC_OFFSETS.key()); + + break; + } + }); + } + + static void validateScanBoundedMode(ReadableConfig tableOptions) { + tableOptions + .getOptional(SCAN_BOUNDED_MODE) + .ifPresent( + mode -> { + switch (mode) { + case TIMESTAMP: + if (!tableOptions + .getOptional(SCAN_BOUNDED_TIMESTAMP_MILLIS) + .isPresent()) { + throw new ValidationException( + String.format( + "'%s' is required in '%s' bounded mode" + + " but missing.", + SCAN_BOUNDED_TIMESTAMP_MILLIS.key(), + ScanBoundedMode.TIMESTAMP)); + } + + break; + case SPECIFIC_OFFSETS: + if (!tableOptions + .getOptional(SCAN_BOUNDED_SPECIFIC_OFFSETS) + .isPresent()) { + throw new ValidationException( + String.format( + "'%s' is required in '%s' bounded mode" + + " but missing.", + SCAN_BOUNDED_SPECIFIC_OFFSETS.key(), + ScanBoundedMode.SPECIFIC_OFFSETS)); + } + if (!isSingleTopic(tableOptions)) { + throw new ValidationException( + "Currently Kafka source only supports specific offset for single topic."); + } + String specificOffsets = + tableOptions.get(SCAN_BOUNDED_SPECIFIC_OFFSETS); + parseSpecificOffsets( + specificOffsets, SCAN_BOUNDED_SPECIFIC_OFFSETS.key()); + break; + } + }); + } + + private static void validateSinkPartitioner(ReadableConfig tableOptions) { + tableOptions + .getOptional(SINK_PARTITIONER) + .ifPresent( + partitioner -> { + if (partitioner.equals(SINK_PARTITIONER_VALUE_ROUND_ROBIN) + && tableOptions.getOptional(KEY_FIELDS).isPresent()) { + throw new ValidationException( + "Currently 'round-robin' partitioner only works when option 'key.fields' is not specified."); + } else if (partitioner.isEmpty()) { + throw new ValidationException( + String.format( + "Option '%s' should be a non-empty string.", + SINK_PARTITIONER.key())); + } + }); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + public static List getSourceTopics(ReadableConfig tableOptions) { + return tableOptions.getOptional(TOPIC).orElse(null); + } + + public static Pattern getSourceTopicPattern(ReadableConfig tableOptions) { + return tableOptions.getOptional(TOPIC_PATTERN).map(Pattern::compile).orElse(null); + } + + private static boolean isSingleTopic(ReadableConfig tableOptions) { + // Option 'topic-pattern' is regarded as multi-topics. + return tableOptions.getOptional(TOPIC).map(t -> t.size() == 1).orElse(false); + } + + public static StartupOptions getStartupOptions(ReadableConfig tableOptions) { + final Map specificOffsets = new HashMap<>(); + final StartupMode startupMode = + tableOptions + .getOptional(SCAN_STARTUP_MODE) + .map(KafkaConnectorOptionsUtil::fromOption) + .orElse(StartupMode.GROUP_OFFSETS); + if (startupMode == StartupMode.SPECIFIC_OFFSETS) { + // It will be refactored after support specific offset for multiple topics in + // FLINK-18602. We have already checked tableOptions.get(TOPIC) contains one topic in + // validateScanStartupMode(). + buildSpecificOffsets(tableOptions, tableOptions.get(TOPIC).get(0), specificOffsets); + } + + final StartupOptions options = new StartupOptions(); + options.startupMode = startupMode; + options.specificOffsets = specificOffsets; + if (startupMode == StartupMode.TIMESTAMP) { + options.startupTimestampMillis = tableOptions.get(SCAN_STARTUP_TIMESTAMP_MILLIS); + } + return options; + } + + public static BoundedOptions getBoundedOptions(ReadableConfig tableOptions) { + final Map specificOffsets = new HashMap<>(); + final BoundedMode boundedMode = + KafkaConnectorOptionsUtil.fromOption(tableOptions.get(SCAN_BOUNDED_MODE)); + if (boundedMode == BoundedMode.SPECIFIC_OFFSETS) { + buildBoundedOffsets(tableOptions, tableOptions.get(TOPIC).get(0), specificOffsets); + } + + final BoundedOptions options = new BoundedOptions(); + options.boundedMode = boundedMode; + options.specificOffsets = specificOffsets; + if (boundedMode == BoundedMode.TIMESTAMP) { + options.boundedTimestampMillis = tableOptions.get(SCAN_BOUNDED_TIMESTAMP_MILLIS); + } + return options; + } + + private static void buildSpecificOffsets( + ReadableConfig tableOptions, + String topic, + Map specificOffsets) { + String specificOffsetsStrOpt = tableOptions.get(SCAN_STARTUP_SPECIFIC_OFFSETS); + final Map offsetMap = + parseSpecificOffsets(specificOffsetsStrOpt, SCAN_STARTUP_SPECIFIC_OFFSETS.key()); + offsetMap.forEach( + (partition, offset) -> { + final KafkaTopicPartition topicPartition = + new KafkaTopicPartition(topic, partition); + specificOffsets.put(topicPartition, offset); + }); + } + + public static void buildBoundedOffsets( + ReadableConfig tableOptions, + String topic, + Map specificOffsets) { + String specificOffsetsEndOpt = tableOptions.get(SCAN_BOUNDED_SPECIFIC_OFFSETS); + final Map offsetMap = + parseSpecificOffsets(specificOffsetsEndOpt, SCAN_BOUNDED_SPECIFIC_OFFSETS.key()); + + offsetMap.forEach( + (partition, offset) -> { + final KafkaTopicPartition topicPartition = + new KafkaTopicPartition(topic, partition); + specificOffsets.put(topicPartition, offset); + }); + } + + /** + * Returns the {@link StartupMode} of Kafka Consumer by passed-in table-specific {@link + * ScanStartupMode}. + */ + private static StartupMode fromOption(ScanStartupMode scanStartupMode) { + switch (scanStartupMode) { + case EARLIEST_OFFSET: + return StartupMode.EARLIEST; + case LATEST_OFFSET: + return StartupMode.LATEST; + case GROUP_OFFSETS: + return StartupMode.GROUP_OFFSETS; + case SPECIFIC_OFFSETS: + return StartupMode.SPECIFIC_OFFSETS; + case TIMESTAMP: + return StartupMode.TIMESTAMP; + + default: + throw new TableException( + "Unsupported startup mode. Validator should have checked that."); + } + } + + /** + * Returns the {@link BoundedMode} of Kafka Consumer by passed-in table-specific {@link + * ScanBoundedMode}. + */ + private static BoundedMode fromOption(ScanBoundedMode scanBoundedMode) { + switch (scanBoundedMode) { + case UNBOUNDED: + return BoundedMode.UNBOUNDED; + case LATEST_OFFSET: + return BoundedMode.LATEST; + case GROUP_OFFSETS: + return BoundedMode.GROUP_OFFSETS; + case TIMESTAMP: + return BoundedMode.TIMESTAMP; + case SPECIFIC_OFFSETS: + return BoundedMode.SPECIFIC_OFFSETS; + + default: + throw new TableException( + "Unsupported bounded mode. Validator should have checked that."); + } + } + + public static Properties getKafkaProperties(Map tableOptions) { + final Properties kafkaProperties = new Properties(); + + if (hasKafkaClientProperties(tableOptions)) { + tableOptions.keySet().stream() + .filter(key -> key.startsWith(PROPERTIES_PREFIX)) + .forEach( + key -> { + final String value = tableOptions.get(key); + final String subKey = key.substring((PROPERTIES_PREFIX).length()); + kafkaProperties.put(subKey, value); + }); + } + return kafkaProperties; + } + + /** + * The partitioner can be either "fixed", "round-robin" or a customized partitioner full class + * name. + */ + public static Optional> getFlinkKafkaPartitioner( + ReadableConfig tableOptions, ClassLoader classLoader) { + return tableOptions + .getOptional(SINK_PARTITIONER) + .flatMap( + (String partitioner) -> { + switch (partitioner) { + case SINK_PARTITIONER_VALUE_FIXED: + return Optional.of(new FlinkFixedPartitioner<>()); + case SINK_PARTITIONER_VALUE_DEFAULT: + case SINK_PARTITIONER_VALUE_ROUND_ROBIN: + return Optional.empty(); + // Default fallback to full class name of the partitioner. + default: + return Optional.of( + initializePartitioner(partitioner, classLoader)); + } + }); + } + + /** + * Parses specificOffsets String to Map. + * + *

specificOffsets String format was given as following: + * + *

+     *     scan.startup.specific-offsets = partition:0,offset:42;partition:1,offset:300
+     * 
+ * + * @return specificOffsets with Map format, key is partition, and value is offset + */ + public static Map parseSpecificOffsets( + String specificOffsetsStr, String optionKey) { + final Map offsetMap = new HashMap<>(); + final String[] pairs = specificOffsetsStr.split(";"); + final String validationExceptionMessage = + String.format( + "Invalid properties '%s' should follow the format " + + "'partition:0,offset:42;partition:1,offset:300', but is '%s'.", + optionKey, specificOffsetsStr); + + if (pairs.length == 0) { + throw new ValidationException(validationExceptionMessage); + } + + for (String pair : pairs) { + if (null == pair || pair.length() == 0 || !pair.contains(",")) { + throw new ValidationException(validationExceptionMessage); + } + + final String[] kv = pair.split(","); + if (kv.length != 2 + || !kv[0].startsWith(PARTITION + ':') + || !kv[1].startsWith(OFFSET + ':')) { + throw new ValidationException(validationExceptionMessage); + } + + String partitionValue = kv[0].substring(kv[0].indexOf(":") + 1); + String offsetValue = kv[1].substring(kv[1].indexOf(":") + 1); + try { + final Integer partition = Integer.valueOf(partitionValue); + final Long offset = Long.valueOf(offsetValue); + offsetMap.put(partition, offset); + } catch (NumberFormatException e) { + throw new ValidationException(validationExceptionMessage, e); + } + } + return offsetMap; + } + + /** + * Decides if the table options contains Kafka client properties that start with prefix + * 'properties'. + */ + private static boolean hasKafkaClientProperties(Map tableOptions) { + return tableOptions.keySet().stream().anyMatch(k -> k.startsWith(PROPERTIES_PREFIX)); + } + + /** Returns a class value with the given class name. */ + private static FlinkKafkaPartitioner initializePartitioner( + String name, ClassLoader classLoader) { + try { + Class clazz = Class.forName(name, true, classLoader); + if (!FlinkKafkaPartitioner.class.isAssignableFrom(clazz)) { + throw new ValidationException( + String.format( + "Sink partitioner class '%s' should extend from the required class %s", + name, FlinkKafkaPartitioner.class.getName())); + } + @SuppressWarnings("unchecked") + final FlinkKafkaPartitioner kafkaPartitioner = + InstantiationUtil.instantiate(name, FlinkKafkaPartitioner.class, classLoader); + + return kafkaPartitioner; + } catch (ClassNotFoundException | FlinkException e) { + throw new ValidationException( + String.format("Could not find and instantiate partitioner class '%s'", name), + e); + } + } + + /** + * Creates an array of indices that determine which physical fields of the table schema to + * include in the key format and the order that those fields have in the key format. + * + *

See {@link KafkaConnectorOptions#KEY_FORMAT}, {@link KafkaConnectorOptions#KEY_FIELDS}, + * and {@link KafkaConnectorOptions#KEY_FIELDS_PREFIX} for more information. + */ + public static int[] createKeyFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + Preconditions.checkArgument( + physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + final Optional optionalKeyFormat = options.getOptional(KEY_FORMAT); + final Optional> optionalKeyFields = options.getOptional(KEY_FIELDS); + + if (!optionalKeyFormat.isPresent() && optionalKeyFields.isPresent()) { + throw new ValidationException( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + } else if (optionalKeyFormat.isPresent() + && (!optionalKeyFields.isPresent() || optionalKeyFields.get().size() == 0)) { + throw new ValidationException( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + } + + if (!optionalKeyFormat.isPresent()) { + return new int[0]; + } + + final String keyPrefix = options.getOptional(KEY_FIELDS_PREFIX).orElse(""); + + final List keyFields = optionalKeyFields.get(); + final List physicalFields = LogicalTypeChecks.getFieldNames(physicalType); + return keyFields.stream() + .mapToInt( + keyField -> { + final int pos = physicalFields.indexOf(keyField); + // check that field name exists + if (pos < 0) { + throw new ValidationException( + String.format( + "Could not find the field '%s' in the table schema for usage in the key format. " + + "A key field must be a regular, physical column. " + + "The following columns can be selected in the '%s' option:\n" + + "%s", + keyField, KEY_FIELDS.key(), physicalFields)); + } + // check that field name is prefixed correctly + if (!keyField.startsWith(keyPrefix)) { + throw new ValidationException( + String.format( + "All fields in '%s' must be prefixed with '%s' when option '%s' " + + "is set but field '%s' is not prefixed.", + KEY_FIELDS.key(), + keyPrefix, + KEY_FIELDS_PREFIX.key(), + keyField)); + } + return pos; + }) + .toArray(); + } + + /** + * Creates an array of indices that determine which physical fields of the table schema to + * include in the value format. + * + *

See {@link KafkaConnectorOptions#VALUE_FORMAT}, {@link + * KafkaConnectorOptions#VALUE_FIELDS_INCLUDE}, and {@link + * KafkaConnectorOptions#KEY_FIELDS_PREFIX} for more information. + */ + public static int[] createValueFormatProjection( + ReadableConfig options, DataType physicalDataType) { + final LogicalType physicalType = physicalDataType.getLogicalType(); + Preconditions.checkArgument( + physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + final int physicalFieldCount = LogicalTypeChecks.getFieldCount(physicalType); + final IntStream physicalFields = IntStream.range(0, physicalFieldCount); + + final String keyPrefix = options.getOptional(KEY_FIELDS_PREFIX).orElse(""); + + final ValueFieldsStrategy strategy = options.get(VALUE_FIELDS_INCLUDE); + if (strategy == ValueFieldsStrategy.ALL) { + if (keyPrefix.length() > 0) { + throw new ValidationException( + String.format( + "A key prefix is not allowed when option '%s' is set to '%s'. " + + "Set it to '%s' instead to avoid field overlaps.", + VALUE_FIELDS_INCLUDE.key(), + ValueFieldsStrategy.ALL, + ValueFieldsStrategy.EXCEPT_KEY)); + } + return physicalFields.toArray(); + } else if (strategy == ValueFieldsStrategy.EXCEPT_KEY) { + final int[] keyProjection = createKeyFormatProjection(options, physicalDataType); + return physicalFields + .filter(pos -> IntStream.of(keyProjection).noneMatch(k -> k == pos)) + .toArray(); + } + throw new TableException("Unknown value fields strategy:" + strategy); + } + + /** + * Returns a new table context with a default schema registry subject value in the options if + * the format is a schema registry format (e.g. 'avro-confluent') and the subject is not + * defined. + */ + public static DynamicTableFactory.Context autoCompleteSchemaRegistrySubject( + DynamicTableFactory.Context context) { + Map tableOptions = context.getCatalogTable().getOptions(); + Map newOptions = autoCompleteSchemaRegistrySubject(tableOptions); + if (newOptions.size() > tableOptions.size()) { + // build a new context + return new FactoryUtil.DefaultDynamicTableContext( + context.getObjectIdentifier(), + context.getCatalogTable().copy(newOptions), + context.getEnrichmentOptions(), + context.getConfiguration(), + context.getClassLoader(), + context.isTemporary()); + } else { + return context; + } + } + + private static Map autoCompleteSchemaRegistrySubject( + Map options) { + Configuration configuration = Configuration.fromMap(options); + // the subject autoComplete should only be used in sink, check the topic first + validateSinkTopic(configuration); + final Optional valueFormat = configuration.getOptional(VALUE_FORMAT); + final Optional keyFormat = configuration.getOptional(KEY_FORMAT); + final Optional format = configuration.getOptional(FORMAT); + final String topic = configuration.get(TOPIC).get(0); + + if (format.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(format.get())) { + autoCompleteSubject(configuration, format.get(), topic + "-value"); + } else if (valueFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(valueFormat.get())) { + autoCompleteSubject(configuration, "value." + valueFormat.get(), topic + "-value"); + } + + if (keyFormat.isPresent() && SCHEMA_REGISTRY_FORMATS.contains(keyFormat.get())) { + autoCompleteSubject(configuration, "key." + keyFormat.get(), topic + "-key"); + } + return configuration.toMap(); + } + + private static void autoCompleteSubject( + Configuration configuration, String format, String subject) { + ConfigOption subjectOption = + ConfigOptions.key(format + "." + SCHEMA_REGISTRY_SUBJECT.key()) + .stringType() + .noDefaultValue(); + if (!configuration.getOptional(subjectOption).isPresent()) { + configuration.setString(subjectOption, subject); + } + } + + static void validateDeliveryGuarantee(ReadableConfig tableOptions) { + if (tableOptions.get(DELIVERY_GUARANTEE) == DeliveryGuarantee.EXACTLY_ONCE + && !tableOptions.getOptional(TRANSACTIONAL_ID_PREFIX).isPresent()) { + throw new ValidationException( + TRANSACTIONAL_ID_PREFIX.key() + + " must be specified when using DeliveryGuarantee.EXACTLY_ONCE."); + } + } + + // -------------------------------------------------------------------------------------------- + // Inner classes + // -------------------------------------------------------------------------------------------- + + /** Kafka startup options. * */ + public static class StartupOptions { + + public StartupMode startupMode; + public Map specificOffsets; + public long startupTimestampMillis; + } + + /** Kafka bounded options. * */ + public static class BoundedOptions { + + public BoundedMode boundedMode; + public Map specificOffsets; + public long boundedTimestampMillis; + } + + private KafkaConnectorOptionsUtil() { + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSink.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSink.java new file mode 100644 index 00000000000..0832a8b81e0 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSink.java @@ -0,0 +1,510 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.sink.KafkaSink; +import org.apache.flink.connector.kafka.sink.KafkaSinkBuilder; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.header.Header; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A version-agnostic Kafka {@link DynamicTableSink}. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +public class KafkaDynamicSink implements DynamicTableSink, SupportsWritingMetadata { + + private static final String UPSERT_KAFKA_TRANSFORMATION = "upsert-kafka"; + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + /** + * Metadata that is appended at the end of a physical sink row. + */ + protected List metadataKeys; + + // -------------------------------------------------------------------------------------------- + // Format attributes + // -------------------------------------------------------------------------------------------- + + /** + * Data type of consumed data type. + */ + protected DataType consumedDataType; + + /** + * Data type to configure the formats. + */ + protected final DataType physicalDataType; + + /** + * Optional format for encoding keys to Kafka. + */ + protected final @Nullable EncodingFormat> keyEncodingFormat; + + /** + * Format for encoding values to Kafka. + */ + protected final EncodingFormat> valueEncodingFormat; + + /** + * Indices that determine the key fields and the source position in the consumed row. + */ + protected final int[] keyProjection; + + /** + * Indices that determine the value fields and the source position in the consumed row. + */ + protected final int[] valueProjection; + + /** + * Prefix that needs to be removed from fields when constructing the physical data type. + */ + protected final @Nullable String keyPrefix; + + // -------------------------------------------------------------------------------------------- + // Kafka-specific attributes + // -------------------------------------------------------------------------------------------- + + /** + * The defined delivery guarantee. + */ + private final DeliveryGuarantee deliveryGuarantee; + + /** + * If the {@link #deliveryGuarantee} is {@link DeliveryGuarantee#EXACTLY_ONCE} the value is the + * prefix for all ids of opened Kafka transactions. + */ + @Nullable + private final String transactionalIdPrefix; + + /** + * The Kafka topic to write to. + */ + protected final String topic; + + /** + * Properties for the Kafka producer. + */ + protected final Properties properties; + + /** + * Partitioner to select Kafka partition for each item. + */ + protected final @Nullable FlinkKafkaPartitioner partitioner; + + /** + * Flag to determine sink mode. In upsert mode sink transforms the delete/update-before message + * to tombstone message. + */ + protected final boolean upsertMode; + + /** + * Sink buffer flush config which only supported in upsert mode now. + */ + protected final SinkBufferFlushMode flushMode; + + /** + * Parallelism of the physical Kafka producer. * + */ + protected final @Nullable Integer parallelism; + + public KafkaDynamicSink( + DataType consumedDataType, + DataType physicalDataType, + @Nullable EncodingFormat> keyEncodingFormat, + EncodingFormat> valueEncodingFormat, + int[] keyProjection, + int[] valueProjection, + @Nullable String keyPrefix, + String topic, + Properties properties, + @Nullable FlinkKafkaPartitioner partitioner, + DeliveryGuarantee deliveryGuarantee, + boolean upsertMode, + SinkBufferFlushMode flushMode, + @Nullable Integer parallelism, + @Nullable String transactionalIdPrefix) { + // Format attributes + this.consumedDataType = + checkNotNull(consumedDataType, "Consumed data type must not be null."); + this.physicalDataType = + checkNotNull(physicalDataType, "Physical data type must not be null."); + this.keyEncodingFormat = keyEncodingFormat; + this.valueEncodingFormat = + checkNotNull(valueEncodingFormat, "Value encoding format must not be null."); + this.keyProjection = checkNotNull(keyProjection, "Key projection must not be null."); + this.valueProjection = checkNotNull(valueProjection, "Value projection must not be null."); + this.keyPrefix = keyPrefix; + this.transactionalIdPrefix = transactionalIdPrefix; + // Mutable attributes + this.metadataKeys = Collections.emptyList(); + // Kafka-specific attributes + this.topic = checkNotNull(topic, "Topic must not be null."); + this.properties = checkNotNull(properties, "Properties must not be null."); + this.partitioner = partitioner; + this.deliveryGuarantee = + checkNotNull(deliveryGuarantee, "DeliveryGuarantee must not be null."); + this.upsertMode = upsertMode; + this.flushMode = checkNotNull(flushMode); + if (flushMode.isEnabled() && !upsertMode) { + throw new IllegalArgumentException( + "Sink buffer flush is only supported in upsert-kafka."); + } + this.parallelism = parallelism; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + return valueEncodingFormat.getChangelogMode(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + final SerializationSchema keySerialization = + createSerialization(context, keyEncodingFormat, keyProjection, keyPrefix); + + final SerializationSchema valueSerialization = + createSerialization(context, valueEncodingFormat, valueProjection, null); + + final KafkaSinkBuilder sinkBuilder = KafkaSink.builder(); + final List physicalChildren = physicalDataType.getLogicalType().getChildren(); + if (transactionalIdPrefix != null) { + sinkBuilder.setTransactionalIdPrefix(transactionalIdPrefix); + } + final KafkaSink kafkaSink = + sinkBuilder + .setDeliveryGuarantee(deliveryGuarantee) + .setBootstrapServers( + properties.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG).toString()) + .setKafkaProducerConfig(properties) + .setRecordSerializer( + new DynamicKafkaRecordSerializationSchema( + topic, + partitioner, + keySerialization, + valueSerialization, + getFieldGetters(physicalChildren, keyProjection), + getFieldGetters(physicalChildren, valueProjection), + hasMetadata(), + getMetadataPositions(physicalChildren), + upsertMode)) + .build(); + if (flushMode.isEnabled() && upsertMode) { + return new DataStreamSinkProvider() { + + @Override + public DataStreamSink consumeDataStream( + ProviderContext providerContext, DataStream dataStream) { + final boolean objectReuse = + dataStream.getExecutionEnvironment().getConfig().isObjectReuseEnabled(); + final ReducingUpsertSink sink = + new ReducingUpsertSink<>( + kafkaSink, + physicalDataType, + keyProjection, + flushMode, + objectReuse + ? createRowDataTypeSerializer( + context, + dataStream.getExecutionConfig())::copy + : rowData -> rowData); + final DataStreamSink end = dataStream.sinkTo(sink); + providerContext.generateUid(UPSERT_KAFKA_TRANSFORMATION).ifPresent(end::uid); + if (parallelism != null) { + end.setParallelism(parallelism); + } + return end; + } + }; + } + return SinkV2Provider.of(kafkaSink, parallelism); + } + + @Override + public Map listWritableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(WritableMetadata.values()) + .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyWritableMetadata(List metadataKeys, DataType consumedDataType) { + this.metadataKeys = metadataKeys; + this.consumedDataType = consumedDataType; + } + + @Override + public DynamicTableSink copy() { + final KafkaDynamicSink copy = + new KafkaDynamicSink( + consumedDataType, + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topic, + properties, + partitioner, + deliveryGuarantee, + upsertMode, + flushMode, + parallelism, + transactionalIdPrefix); + copy.metadataKeys = metadataKeys; + return copy; + } + + @Override + public String asSummaryString() { + return "Kafka table sink"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final KafkaDynamicSink that = (KafkaDynamicSink) o; + return Objects.equals(metadataKeys, that.metadataKeys) + && Objects.equals(consumedDataType, that.consumedDataType) + && Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyEncodingFormat, that.keyEncodingFormat) + && Objects.equals(valueEncodingFormat, that.valueEncodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(keyPrefix, that.keyPrefix) + && Objects.equals(topic, that.topic) + && Objects.equals(properties, that.properties) + && Objects.equals(partitioner, that.partitioner) + && Objects.equals(deliveryGuarantee, that.deliveryGuarantee) + && Objects.equals(upsertMode, that.upsertMode) + && Objects.equals(flushMode, that.flushMode) + && Objects.equals(transactionalIdPrefix, that.transactionalIdPrefix) + && Objects.equals(parallelism, that.parallelism); + } + + @Override + public int hashCode() { + return Objects.hash( + metadataKeys, + consumedDataType, + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topic, + properties, + partitioner, + deliveryGuarantee, + upsertMode, + flushMode, + transactionalIdPrefix, + parallelism); + } + + // -------------------------------------------------------------------------------------------- + + private TypeSerializer createRowDataTypeSerializer( + Context context, ExecutionConfig executionConfig) { + final TypeInformation typeInformation = + context.createTypeInformation(consumedDataType); + return typeInformation.createSerializer(executionConfig); + } + + private int[] getMetadataPositions(List physicalChildren) { + return Stream.of(WritableMetadata.values()) + .mapToInt( + m -> { + final int pos = metadataKeys.indexOf(m.key); + if (pos < 0) { + return -1; + } + return physicalChildren.size() + pos; + }) + .toArray(); + } + + private boolean hasMetadata() { + return metadataKeys.size() > 0; + } + + private RowData.FieldGetter[] getFieldGetters( + List physicalChildren, int[] keyProjection) { + return Arrays.stream(keyProjection) + .mapToObj( + targetField -> RowData.createFieldGetter( + physicalChildren.get(targetField), targetField)) + .toArray(RowData.FieldGetter[]::new); + } + + private @Nullable SerializationSchema createSerialization( + DynamicTableSink.Context context, + @Nullable EncodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeEncoder(context, physicalFormatDataType); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + enum WritableMetadata { + + HEADERS( + "headers", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.BYTES().nullable()) + .nullable(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(RowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + final MapData map = row.getMap(pos); + final ArrayData keyArray = map.keyArray(); + final ArrayData valueArray = map.valueArray(); + final List

headers = new ArrayList<>(); + for (int i = 0; i < keyArray.size(); i++) { + if (!keyArray.isNullAt(i) && !valueArray.isNullAt(i)) { + final String key = keyArray.getString(i).toString(); + final byte[] value = valueArray.getBinary(i); + headers.add(new KafkaHeader(key, value)); + } + } + return headers; + } + }), + + TIMESTAMP( + "timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + new MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(RowData row, int pos) { + if (row.isNullAt(pos)) { + return null; + } + return row.getTimestamp(pos, 3).getMillisecond(); + } + }); + + final String key; + + final DataType dataType; + + final MetadataConverter converter; + + WritableMetadata(String key, DataType dataType, MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } + + interface MetadataConverter extends Serializable { + + Object read(RowData consumedRow, int pos); + } + + // -------------------------------------------------------------------------------------------- + + private static class KafkaHeader implements Header { + + private final String key; + + private final byte[] value; + + KafkaHeader(String key, byte[] value) { + this.key = key; + this.value = value; + } + + @Override + public String key() { + return key; + } + + @Override + public byte[] value() { + return value; + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSource.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSource.java new file mode 100644 index 00000000000..1d963ea2582 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSource.java @@ -0,0 +1,713 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.inlong.sort.kafka.source.KafkaSource; +import org.apache.inlong.sort.kafka.source.KafkaSourceBuilder; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.connector.kafka.source.enumerator.initializer.NoStoppingOffsetsInitializer; +import org.apache.flink.connector.kafka.source.enumerator.initializer.OffsetsInitializer; +import org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.connectors.kafka.KafkaDeserializationSchema; +import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata; +import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.util.Preconditions; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Header; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +/** + * A version-agnostic Kafka {@link ScanTableSource}. + * org.apache.flink:flink-connector-kafka:3.2.0 + */ +public class KafkaDynamicSource + implements + ScanTableSource, + SupportsReadingMetadata, + SupportsWatermarkPushDown { + + private static final String KAFKA_TRANSFORMATION = "kafka"; + + // -------------------------------------------------------------------------------------------- + // Mutable attributes + // -------------------------------------------------------------------------------------------- + + /** + * Data type that describes the final output of the source. + */ + protected DataType producedDataType; + + /** + * Metadata that is appended at the end of a physical source row. + */ + protected List metadataKeys; + + /** + * Watermark strategy that is used to generate per-partition watermark. + */ + protected @Nullable WatermarkStrategy watermarkStrategy; + + // -------------------------------------------------------------------------------------------- + // Format attributes + // -------------------------------------------------------------------------------------------- + + private static final String VALUE_METADATA_PREFIX = "value."; + + /** + * Data type to configure the formats. + */ + protected final DataType physicalDataType; + + /** + * Optional format for decoding keys from Kafka. + */ + protected final @Nullable DecodingFormat> keyDecodingFormat; + + /** + * Format for decoding values from Kafka. + */ + protected final DecodingFormat> valueDecodingFormat; + + /** + * Indices that determine the key fields and the target position in the produced row. + */ + protected final int[] keyProjection; + + /** + * Indices that determine the value fields and the target position in the produced row. + */ + protected final int[] valueProjection; + + /** + * Prefix that needs to be removed from fields when constructing the physical data type. + */ + protected final @Nullable String keyPrefix; + + // -------------------------------------------------------------------------------------------- + // Kafka-specific attributes + // -------------------------------------------------------------------------------------------- + + /** + * The Kafka topics to consume. + */ + protected final List topics; + + /** + * The Kafka topic pattern to consume. + */ + protected final Pattern topicPattern; + + /** + * Properties for the Kafka consumer. + */ + protected final Properties properties; + + /** + * The startup mode for the contained consumer (default is {@link StartupMode#GROUP_OFFSETS}). + */ + protected final StartupMode startupMode; + + /** + * Specific startup offsets; only relevant when startup mode is {@link + * StartupMode#SPECIFIC_OFFSETS}. + */ + protected final Map specificStartupOffsets; + + /** + * The start timestamp to locate partition offsets; only relevant when startup mode is {@link + * StartupMode#TIMESTAMP}. + */ + protected final long startupTimestampMillis; + + /** + * The bounded mode for the contained consumer (default is an unbounded data stream). + */ + protected final BoundedMode boundedMode; + + /** + * Specific end offsets; only relevant when bounded mode is {@link + * BoundedMode#SPECIFIC_OFFSETS}. + */ + protected final Map specificBoundedOffsets; + + /** + * The bounded timestamp to locate partition offsets; only relevant when bounded mode is {@link + * BoundedMode#TIMESTAMP}. + */ + protected final long boundedTimestampMillis; + + /** + * Flag to determine source mode. In upsert mode, it will keep the tombstone message. * + */ + protected final boolean upsertMode; + + protected final String tableIdentifier; + + public KafkaDynamicSource( + DataType physicalDataType, + @Nullable DecodingFormat> keyDecodingFormat, + DecodingFormat> valueDecodingFormat, + int[] keyProjection, + int[] valueProjection, + @Nullable String keyPrefix, + @Nullable List topics, + @Nullable Pattern topicPattern, + Properties properties, + StartupMode startupMode, + Map specificStartupOffsets, + long startupTimestampMillis, + BoundedMode boundedMode, + Map specificBoundedOffsets, + long boundedTimestampMillis, + boolean upsertMode, + String tableIdentifier) { + // Format attributes + this.physicalDataType = + Preconditions.checkNotNull( + physicalDataType, "Physical data type must not be null."); + this.keyDecodingFormat = keyDecodingFormat; + this.valueDecodingFormat = + Preconditions.checkNotNull( + valueDecodingFormat, "Value decoding format must not be null."); + this.keyProjection = + Preconditions.checkNotNull(keyProjection, "Key projection must not be null."); + this.valueProjection = + Preconditions.checkNotNull(valueProjection, "Value projection must not be null."); + this.keyPrefix = keyPrefix; + // Mutable attributes + this.producedDataType = physicalDataType; + this.metadataKeys = Collections.emptyList(); + this.watermarkStrategy = null; + // Kafka-specific attributes + Preconditions.checkArgument( + (topics != null && topicPattern == null) + || (topics == null && topicPattern != null), + "Either Topic or Topic Pattern must be set for source."); + this.topics = topics; + this.topicPattern = topicPattern; + this.properties = Preconditions.checkNotNull(properties, "Properties must not be null."); + this.startupMode = + Preconditions.checkNotNull(startupMode, "Startup mode must not be null."); + this.specificStartupOffsets = + Preconditions.checkNotNull( + specificStartupOffsets, "Specific offsets must not be null."); + this.startupTimestampMillis = startupTimestampMillis; + this.boundedMode = + Preconditions.checkNotNull(boundedMode, "Bounded mode must not be null."); + this.specificBoundedOffsets = + Preconditions.checkNotNull( + specificBoundedOffsets, "Specific bounded offsets must not be null."); + this.boundedTimestampMillis = boundedTimestampMillis; + this.upsertMode = upsertMode; + this.tableIdentifier = tableIdentifier; + } + + @Override + public ChangelogMode getChangelogMode() { + return valueDecodingFormat.getChangelogMode(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { + final DeserializationSchema keyDeserialization = + createDeserialization(context, keyDecodingFormat, keyProjection, keyPrefix); + + final DeserializationSchema valueDeserialization = + createDeserialization(context, valueDecodingFormat, valueProjection, null); + + final TypeInformation producedTypeInfo = + context.createTypeInformation(producedDataType); + + final KafkaSource kafkaSource = + createKafkaSource(keyDeserialization, valueDeserialization, producedTypeInfo); + + return new DataStreamScanProvider() { + + @Override + public DataStream produceDataStream( + ProviderContext providerContext, StreamExecutionEnvironment execEnv) { + if (watermarkStrategy == null) { + watermarkStrategy = WatermarkStrategy.noWatermarks(); + } + DataStreamSource sourceStream = + execEnv.fromSource( + kafkaSource, watermarkStrategy, "KafkaSource-" + tableIdentifier); + providerContext.generateUid(KAFKA_TRANSFORMATION).ifPresent(sourceStream::uid); + return sourceStream; + } + + @Override + public boolean isBounded() { + return kafkaSource.getBoundedness() == Boundedness.BOUNDED; + } + }; + } + + @Override + public Map listReadableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + + // according to convention, the order of the final row must be + // PHYSICAL + FORMAT METADATA + CONNECTOR METADATA + // where the format metadata has highest precedence + + // add value format metadata with prefix + valueDecodingFormat + .listReadableMetadata() + .forEach((key, value) -> metadataMap.put(VALUE_METADATA_PREFIX + key, value)); + + // add connector metadata + Stream.of(ReadableMetadata.values()) + .forEachOrdered(m -> metadataMap.putIfAbsent(m.key, m.dataType)); + + return metadataMap; + } + + @Override + public void applyReadableMetadata(List metadataKeys, DataType producedDataType) { + // separate connector and format metadata + final List formatMetadataKeys = + metadataKeys.stream() + .filter(k -> k.startsWith(VALUE_METADATA_PREFIX)) + .collect(Collectors.toList()); + final List connectorMetadataKeys = new ArrayList<>(metadataKeys); + connectorMetadataKeys.removeAll(formatMetadataKeys); + + // push down format metadata + final Map formatMetadata = valueDecodingFormat.listReadableMetadata(); + if (formatMetadata.size() > 0) { + final List requestedFormatMetadataKeys = + formatMetadataKeys.stream() + .map(k -> k.substring(VALUE_METADATA_PREFIX.length())) + .collect(Collectors.toList()); + valueDecodingFormat.applyReadableMetadata(requestedFormatMetadataKeys); + } + + this.metadataKeys = connectorMetadataKeys; + this.producedDataType = producedDataType; + } + + @Override + public boolean supportsMetadataProjection() { + return false; + } + + @Override + public void applyWatermark(WatermarkStrategy watermarkStrategy) { + this.watermarkStrategy = watermarkStrategy; + } + + @Override + public DynamicTableSource copy() { + final KafkaDynamicSource copy = + new KafkaDynamicSource( + physicalDataType, + keyDecodingFormat, + valueDecodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topics, + topicPattern, + properties, + startupMode, + specificStartupOffsets, + startupTimestampMillis, + boundedMode, + specificBoundedOffsets, + boundedTimestampMillis, + upsertMode, + tableIdentifier); + copy.producedDataType = producedDataType; + copy.metadataKeys = metadataKeys; + copy.watermarkStrategy = watermarkStrategy; + return copy; + } + + @Override + public String asSummaryString() { + return "Kafka table source"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final KafkaDynamicSource that = (KafkaDynamicSource) o; + return Objects.equals(producedDataType, that.producedDataType) + && Objects.equals(metadataKeys, that.metadataKeys) + && Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyDecodingFormat, that.keyDecodingFormat) + && Objects.equals(valueDecodingFormat, that.valueDecodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(keyPrefix, that.keyPrefix) + && Objects.equals(topics, that.topics) + && Objects.equals(String.valueOf(topicPattern), String.valueOf(that.topicPattern)) + && Objects.equals(properties, that.properties) + && startupMode == that.startupMode + && Objects.equals(specificStartupOffsets, that.specificStartupOffsets) + && startupTimestampMillis == that.startupTimestampMillis + && boundedMode == that.boundedMode + && Objects.equals(specificBoundedOffsets, that.specificBoundedOffsets) + && boundedTimestampMillis == that.boundedTimestampMillis + && Objects.equals(upsertMode, that.upsertMode) + && Objects.equals(tableIdentifier, that.tableIdentifier) + && Objects.equals(watermarkStrategy, that.watermarkStrategy); + } + + @Override + public int hashCode() { + return Objects.hash( + producedDataType, + metadataKeys, + physicalDataType, + keyDecodingFormat, + valueDecodingFormat, + Arrays.hashCode(keyProjection), + Arrays.hashCode(valueProjection), + keyPrefix, + topics, + topicPattern, + properties, + startupMode, + specificStartupOffsets, + startupTimestampMillis, + boundedMode, + specificBoundedOffsets, + boundedTimestampMillis, + upsertMode, + tableIdentifier, + watermarkStrategy); + } + + // -------------------------------------------------------------------------------------------- + + protected KafkaSource createKafkaSource( + DeserializationSchema keyDeserialization, + DeserializationSchema valueDeserialization, + TypeInformation producedTypeInfo) { + + final KafkaDeserializationSchema kafkaDeserializer = + createKafkaDeserializationSchema( + keyDeserialization, valueDeserialization, producedTypeInfo); + + final KafkaSourceBuilder kafkaSourceBuilder = KafkaSource.builder(); + + if (topics != null) { + kafkaSourceBuilder.setTopics(topics); + } else { + kafkaSourceBuilder.setTopicPattern(topicPattern); + } + + switch (startupMode) { + case EARLIEST: + kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.earliest()); + break; + case LATEST: + kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.latest()); + break; + case GROUP_OFFSETS: + String offsetResetConfig = + properties.getProperty( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + OffsetResetStrategy.NONE.name()); + OffsetResetStrategy offsetResetStrategy = getResetStrategy(offsetResetConfig); + kafkaSourceBuilder.setStartingOffsets( + OffsetsInitializer.committedOffsets(offsetResetStrategy)); + break; + case SPECIFIC_OFFSETS: + Map offsets = new HashMap<>(); + specificStartupOffsets.forEach( + (tp, offset) -> offsets.put( + new TopicPartition(tp.getTopic(), tp.getPartition()), + offset)); + kafkaSourceBuilder.setStartingOffsets(OffsetsInitializer.offsets(offsets)); + break; + case TIMESTAMP: + kafkaSourceBuilder.setStartingOffsets( + OffsetsInitializer.timestamp(startupTimestampMillis)); + break; + } + + switch (boundedMode) { + case UNBOUNDED: + kafkaSourceBuilder.setUnbounded(new NoStoppingOffsetsInitializer()); + break; + case LATEST: + kafkaSourceBuilder.setBounded(OffsetsInitializer.latest()); + break; + case GROUP_OFFSETS: + kafkaSourceBuilder.setBounded(OffsetsInitializer.committedOffsets()); + break; + case SPECIFIC_OFFSETS: + Map offsets = new HashMap<>(); + specificBoundedOffsets.forEach( + (tp, offset) -> offsets.put( + new TopicPartition(tp.getTopic(), tp.getPartition()), + offset)); + kafkaSourceBuilder.setBounded(OffsetsInitializer.offsets(offsets)); + break; + case TIMESTAMP: + kafkaSourceBuilder.setBounded(OffsetsInitializer.timestamp(boundedTimestampMillis)); + break; + } + + kafkaSourceBuilder + .setProperties(properties) + .setDeserializer(KafkaRecordDeserializationSchema.of(kafkaDeserializer)); + + return kafkaSourceBuilder.build(); + } + + private OffsetResetStrategy getResetStrategy(String offsetResetConfig) { + return Arrays.stream(OffsetResetStrategy.values()) + .filter(ors -> ors.name().equals(offsetResetConfig.toUpperCase(Locale.ROOT))) + .findAny() + .orElseThrow( + () -> new IllegalArgumentException( + String.format( + "%s can not be set to %s. Valid values: [%s]", + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + offsetResetConfig, + Arrays.stream(OffsetResetStrategy.values()) + .map(Enum::name) + .map(String::toLowerCase) + .collect(Collectors.joining(","))))); + } + + private KafkaDeserializationSchema createKafkaDeserializationSchema( + DeserializationSchema keyDeserialization, + DeserializationSchema valueDeserialization, + TypeInformation producedTypeInfo) { + final DynamicKafkaDeserializationSchema.MetadataConverter[] metadataConverters = + metadataKeys.stream() + .map( + k -> Stream.of(ReadableMetadata.values()) + .filter(rm -> rm.key.equals(k)) + .findFirst() + .orElseThrow(IllegalStateException::new)) + .map(m -> m.converter) + .toArray(DynamicKafkaDeserializationSchema.MetadataConverter[]::new); + + // check if connector metadata is used at all + final boolean hasMetadata = metadataKeys.size() > 0; + + // adjust physical arity with value format's metadata + final int adjustedPhysicalArity = + DataType.getFieldDataTypes(producedDataType).size() - metadataKeys.size(); + + // adjust value format projection to include value format's metadata columns at the end + final int[] adjustedValueProjection = + IntStream.concat( + IntStream.of(valueProjection), + IntStream.range( + keyProjection.length + valueProjection.length, + adjustedPhysicalArity)) + .toArray(); + + return new DynamicKafkaDeserializationSchema( + adjustedPhysicalArity, + keyDeserialization, + keyProjection, + valueDeserialization, + adjustedValueProjection, + hasMetadata, + metadataConverters, + producedTypeInfo, + upsertMode); + } + + private @Nullable DeserializationSchema createDeserialization( + DynamicTableSource.Context context, + @Nullable DecodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeDecoder(context, physicalFormatDataType); + } + + // -------------------------------------------------------------------------------------------- + // Metadata handling + // -------------------------------------------------------------------------------------------- + + enum ReadableMetadata { + + TOPIC( + "topic", + DataTypes.STRING().notNull(), + new DynamicKafkaDeserializationSchema.MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return StringData.fromString(record.topic()); + } + }), + + PARTITION( + "partition", + DataTypes.INT().notNull(), + new DynamicKafkaDeserializationSchema.MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return record.partition(); + } + }), + + HEADERS( + "headers", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.BYTES().nullable()) + .notNull(), + new DynamicKafkaDeserializationSchema.MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + final Map map = new HashMap<>(); + for (Header header : record.headers()) { + map.put(StringData.fromString(header.key()), header.value()); + } + return new GenericMapData(map); + } + }), + + LEADER_EPOCH( + "leader-epoch", + DataTypes.INT().nullable(), + new DynamicKafkaDeserializationSchema.MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return record.leaderEpoch().orElse(null); + } + }), + + OFFSET( + "offset", + DataTypes.BIGINT().notNull(), + new DynamicKafkaDeserializationSchema.MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return record.offset(); + } + }), + + TIMESTAMP( + "timestamp", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).notNull(), + new DynamicKafkaDeserializationSchema.MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return TimestampData.fromEpochMillis(record.timestamp()); + } + }), + + TIMESTAMP_TYPE( + "timestamp-type", + DataTypes.STRING().notNull(), + new DynamicKafkaDeserializationSchema.MetadataConverter() { + + private static final long serialVersionUID = 1L; + + @Override + public Object read(ConsumerRecord record) { + return StringData.fromString(record.timestampType().toString()); + } + }); + + final String key; + + final DataType dataType; + + final DynamicKafkaDeserializationSchema.MetadataConverter converter; + + ReadableMetadata(String key, DataType dataType, DynamicKafkaDeserializationSchema.MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicTableFactory.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicTableFactory.java new file mode 100644 index 00000000000..fb4c69ce297 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicTableFactory.java @@ -0,0 +1,445 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.kafka.source.KafkaSourceOptions; +import org.apache.flink.streaming.connectors.kafka.config.BoundedMode; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicPartition; +import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.format.Format; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.FactoryUtil.TableFactoryHelper; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.RowKind; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_GROUP_ID; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_TOPIC_PARTITION_DISCOVERY; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARTITIONER; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC_PATTERN; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.autoCompleteSchemaRegistrySubject; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.createKeyFormatProjection; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getFlinkKafkaPartitioner; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getSourceTopicPattern; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getSourceTopics; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getStartupOptions; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.validateTableSinkOptions; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.validateTableSourceOptions; +import static org.apache.inlong.sort.protocol.constant.KafkaConstant.KAFKA; + +/** + * Factory for creating configured instances of {@link KafkaDynamicSource} and {@link + * KafkaDynamicSink}. + * org.apache.flink:flink-connector-kafka:3.2.0 + */ +public class KafkaDynamicTableFactory + implements + DynamicTableSourceFactory, + DynamicTableSinkFactory { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaDynamicTableFactory.class); + private static final ConfigOption SINK_SEMANTIC = + ConfigOptions.key("sink.semantic") + .stringType() + .noDefaultValue() + .withDescription("Optional semantic when committing."); + + public static final String IDENTIFIER = KAFKA; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + final Set> options = new HashSet<>(); + options.add(PROPS_BOOTSTRAP_SERVERS); + return options; + } + + @Override + public Set> optionalOptions() { + final Set> options = new HashSet<>(); + options.add(FactoryUtil.FORMAT); + options.add(KEY_FORMAT); + options.add(KEY_FIELDS); + options.add(KEY_FIELDS_PREFIX); + options.add(VALUE_FORMAT); + options.add(VALUE_FIELDS_INCLUDE); + options.add(TOPIC); + options.add(TOPIC_PATTERN); + options.add(PROPS_GROUP_ID); + options.add(SCAN_STARTUP_MODE); + options.add(SCAN_STARTUP_SPECIFIC_OFFSETS); + options.add(SCAN_TOPIC_PARTITION_DISCOVERY); + options.add(SCAN_STARTUP_TIMESTAMP_MILLIS); + options.add(SINK_PARTITIONER); + options.add(SINK_PARALLELISM); + options.add(DELIVERY_GUARANTEE); + options.add(TRANSACTIONAL_ID_PREFIX); + options.add(SINK_SEMANTIC); + options.add(SCAN_BOUNDED_MODE); + options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS); + options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); + return options; + } + + @Override + public Set> forwardOptions() { + return Stream.of( + PROPS_BOOTSTRAP_SERVERS, + PROPS_GROUP_ID, + TOPIC, + TOPIC_PATTERN, + SCAN_STARTUP_MODE, + SCAN_STARTUP_SPECIFIC_OFFSETS, + SCAN_TOPIC_PARTITION_DISCOVERY, + SCAN_STARTUP_TIMESTAMP_MILLIS, + SINK_PARTITIONER, + SINK_PARALLELISM, + TRANSACTIONAL_ID_PREFIX) + .collect(Collectors.toSet()); + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + final TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + + final Optional>> keyDecodingFormat = + getKeyDecodingFormat(helper); + + final DecodingFormat> valueDecodingFormat = + getValueDecodingFormat(helper); + + helper.validateExcept(PROPERTIES_PREFIX); + + final ReadableConfig tableOptions = helper.getOptions(); + + validateTableSourceOptions(tableOptions); + + validatePKConstraints( + context.getObjectIdentifier(), + context.getPrimaryKeyIndexes(), + context.getCatalogTable().getOptions(), + valueDecodingFormat); + + final KafkaConnectorOptionsUtil.StartupOptions startupOptions = getStartupOptions(tableOptions); + + final KafkaConnectorOptionsUtil.BoundedOptions boundedOptions = getBoundedOptions(tableOptions); + + final Properties properties = getKafkaProperties(context.getCatalogTable().getOptions()); + + // add topic-partition discovery + final Duration partitionDiscoveryInterval = + tableOptions.get(SCAN_TOPIC_PARTITION_DISCOVERY); + properties.setProperty( + KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), + Long.toString(partitionDiscoveryInterval.toMillis())); + + final DataType physicalDataType = context.getPhysicalRowDataType(); + + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + + final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + + return createKafkaTableSource( + physicalDataType, + keyDecodingFormat.orElse(null), + valueDecodingFormat, + keyProjection, + valueProjection, + keyPrefix, + getSourceTopics(tableOptions), + getSourceTopicPattern(tableOptions), + properties, + startupOptions.startupMode, + startupOptions.specificOffsets, + startupOptions.startupTimestampMillis, + boundedOptions.boundedMode, + boundedOptions.specificOffsets, + boundedOptions.boundedTimestampMillis, + context.getObjectIdentifier().asSummaryString()); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + final TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper( + this, autoCompleteSchemaRegistrySubject(context)); + + final Optional>> keyEncodingFormat = + getKeyEncodingFormat(helper); + + final EncodingFormat> valueEncodingFormat = + getValueEncodingFormat(helper); + + helper.validateExcept(PROPERTIES_PREFIX); + + final ReadableConfig tableOptions = helper.getOptions(); + + final DeliveryGuarantee deliveryGuarantee = validateDeprecatedSemantic(tableOptions); + validateTableSinkOptions(tableOptions); + + KafkaConnectorOptionsUtil.validateDeliveryGuarantee(tableOptions); + + validatePKConstraints( + context.getObjectIdentifier(), + context.getPrimaryKeyIndexes(), + context.getCatalogTable().getOptions(), + valueEncodingFormat); + + final DataType physicalDataType = context.getPhysicalRowDataType(); + + final int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + + final int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + + final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + + final Integer parallelism = tableOptions.getOptional(SINK_PARALLELISM).orElse(null); + + return createKafkaTableSink( + physicalDataType, + keyEncodingFormat.orElse(null), + valueEncodingFormat, + keyProjection, + valueProjection, + keyPrefix, + tableOptions.get(TOPIC).get(0), + getKafkaProperties(context.getCatalogTable().getOptions()), + getFlinkKafkaPartitioner(tableOptions, context.getClassLoader()).orElse(null), + deliveryGuarantee, + parallelism, + tableOptions.get(TRANSACTIONAL_ID_PREFIX)); + } + + // -------------------------------------------------------------------------------------------- + + private static Optional>> getKeyDecodingFormat( + TableFactoryHelper helper) { + final Optional>> keyDecodingFormat = + helper.discoverOptionalDecodingFormat( + DeserializationFormatFactory.class, KEY_FORMAT); + keyDecodingFormat.ifPresent( + format -> { + if (!format.getChangelogMode().containsOnly(RowKind.INSERT)) { + throw new ValidationException( + String.format( + "A key format should only deal with INSERT-only records. " + + "But %s has a changelog mode of %s.", + helper.getOptions().get(KEY_FORMAT), + format.getChangelogMode())); + } + }); + return keyDecodingFormat; + } + + private static Optional>> getKeyEncodingFormat( + TableFactoryHelper helper) { + final Optional>> keyEncodingFormat = + helper.discoverOptionalEncodingFormat(SerializationFormatFactory.class, KEY_FORMAT); + keyEncodingFormat.ifPresent( + format -> { + if (!format.getChangelogMode().containsOnly(RowKind.INSERT)) { + throw new ValidationException( + String.format( + "A key format should only deal with INSERT-only records. " + + "But %s has a changelog mode of %s.", + helper.getOptions().get(KEY_FORMAT), + format.getChangelogMode())); + } + }); + return keyEncodingFormat; + } + + private static DecodingFormat> getValueDecodingFormat( + TableFactoryHelper helper) { + return helper.discoverOptionalDecodingFormat( + DeserializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> helper.discoverDecodingFormat( + DeserializationFormatFactory.class, VALUE_FORMAT)); + } + + private static EncodingFormat> getValueEncodingFormat( + TableFactoryHelper helper) { + return helper.discoverOptionalEncodingFormat( + SerializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> helper.discoverEncodingFormat( + SerializationFormatFactory.class, VALUE_FORMAT)); + } + + private static void validatePKConstraints( + ObjectIdentifier tableName, + int[] primaryKeyIndexes, + Map options, + Format format) { + if (primaryKeyIndexes.length > 0 + && format.getChangelogMode().containsOnly(RowKind.INSERT)) { + Configuration configuration = Configuration.fromMap(options); + String formatName = + configuration + .getOptional(FactoryUtil.FORMAT) + .orElse(configuration.get(VALUE_FORMAT)); + throw new ValidationException( + String.format( + "The Kafka table '%s' with '%s' format doesn't support defining PRIMARY KEY constraint" + + " on the table, because it can't guarantee the semantic of primary key.", + tableName.asSummaryString(), formatName)); + } + } + + private static DeliveryGuarantee validateDeprecatedSemantic(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SINK_SEMANTIC).isPresent()) { + LOG.warn( + "{} is deprecated and will be removed. Please use {} instead.", + SINK_SEMANTIC.key(), + DELIVERY_GUARANTEE.key()); + return DeliveryGuarantee.valueOf( + tableOptions.get(SINK_SEMANTIC).toUpperCase().replace("-", "_")); + } + return tableOptions.get(DELIVERY_GUARANTEE); + } + + // -------------------------------------------------------------------------------------------- + + protected KafkaDynamicSource createKafkaTableSource( + DataType physicalDataType, + @Nullable DecodingFormat> keyDecodingFormat, + DecodingFormat> valueDecodingFormat, + int[] keyProjection, + int[] valueProjection, + @Nullable String keyPrefix, + @Nullable List topics, + @Nullable Pattern topicPattern, + Properties properties, + StartupMode startupMode, + Map specificStartupOffsets, + long startupTimestampMillis, + BoundedMode boundedMode, + Map specificEndOffsets, + long endTimestampMillis, + String tableIdentifier) { + return new KafkaDynamicSource( + physicalDataType, + keyDecodingFormat, + valueDecodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topics, + topicPattern, + properties, + startupMode, + specificStartupOffsets, + startupTimestampMillis, + boundedMode, + specificEndOffsets, + endTimestampMillis, + false, + tableIdentifier); + } + + protected KafkaDynamicSink createKafkaTableSink( + DataType physicalDataType, + @Nullable EncodingFormat> keyEncodingFormat, + EncodingFormat> valueEncodingFormat, + int[] keyProjection, + int[] valueProjection, + @Nullable String keyPrefix, + String topic, + Properties properties, + FlinkKafkaPartitioner partitioner, + DeliveryGuarantee deliveryGuarantee, + Integer parallelism, + @Nullable String transactionalIdPrefix) { + return new KafkaDynamicSink( + physicalDataType, + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + keyProjection, + valueProjection, + keyPrefix, + topic, + properties, + partitioner, + deliveryGuarantee, + false, + SinkBufferFlushMode.DISABLED, + parallelism, + transactionalIdPrefix); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertSink.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertSink.java new file mode 100644 index 00000000000..97f72376324 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertSink.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.connector.sink.SinkWriter; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.function.SerializableFunction; + +import java.io.IOException; +import java.util.Collection; + +/** + * A wrapper of a {@link Sink}. It will buffer the data emitted by the wrapper {@link SinkWriter} + * and only emit it when the buffer is full or a timer is triggered or a checkpoint happens. + * + *

The sink provides eventual consistency guarantees under {@link + * org.apache.flink.connector.base.DeliveryGuarantee#AT_LEAST_ONCE} because the updates are + * idempotent therefore duplicates have no effect. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +class ReducingUpsertSink + implements + TwoPhaseCommittingStatefulSink { + + private final TwoPhaseCommittingStatefulSink wrappedSink; + private final DataType physicalDataType; + private final int[] keyProjection; + private final SinkBufferFlushMode bufferFlushMode; + private final SerializableFunction valueCopyFunction; + + ReducingUpsertSink( + TwoPhaseCommittingStatefulSink wrappedSink, + DataType physicalDataType, + int[] keyProjection, + SinkBufferFlushMode bufferFlushMode, + SerializableFunction valueCopyFunction) { + this.wrappedSink = wrappedSink; + this.physicalDataType = physicalDataType; + this.keyProjection = keyProjection; + this.bufferFlushMode = bufferFlushMode; + this.valueCopyFunction = valueCopyFunction; + } + + @Override + public TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter createWriter( + InitContext context) throws IOException { + return new ReducingUpsertWriter<>( + wrappedSink.createWriter(context), + physicalDataType, + keyProjection, + bufferFlushMode, + context.getProcessingTimeService(), + valueCopyFunction); + } + + @Override + public Committer createCommitter() throws IOException { + return wrappedSink.createCommitter(); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return wrappedSink.getCommittableSerializer(); + } + + @Override + public TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter restoreWriter( + InitContext context, Collection recoveredState) + throws IOException { + final TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter wrappedWriter = + wrappedSink.restoreWriter(context, recoveredState); + return new ReducingUpsertWriter<>( + wrappedWriter, + physicalDataType, + keyProjection, + bufferFlushMode, + context.getProcessingTimeService(), + valueCopyFunction); + } + + @Override + public SimpleVersionedSerializer getWriterStateSerializer() { + return wrappedSink.getWriterStateSerializer(); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertWriter.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertWriter.java new file mode 100644 index 00000000000..05b4ad6ebc1 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertWriter.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.connector.kafka.sink.TwoPhaseCommittingStatefulSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +import static org.apache.flink.types.RowKind.DELETE; +import static org.apache.flink.types.RowKind.UPDATE_AFTER; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.inlong.sort.kafka.table.DynamicKafkaRecordSerializationSchema.createProjectedRow; + +/** + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +class ReducingUpsertWriter + implements + TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter { + + private final TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter wrappedWriter; + private final WrappedContext wrappedContext = new WrappedContext(); + private final int batchMaxRowNums; + private final Function valueCopyFunction; + private final Map> reduceBuffer = new HashMap<>(); + private final Function keyExtractor; + private final ProcessingTimeService timeService; + private final long batchIntervalMs; + + private boolean closed = false; + private long lastFlush = System.currentTimeMillis(); + + ReducingUpsertWriter( + TwoPhaseCommittingStatefulSink.PrecommittingStatefulSinkWriter wrappedWriter, + DataType physicalDataType, + int[] keyProjection, + SinkBufferFlushMode bufferFlushMode, + ProcessingTimeService timeService, + Function valueCopyFunction) { + checkArgument(bufferFlushMode != null && bufferFlushMode.isEnabled()); + this.wrappedWriter = checkNotNull(wrappedWriter); + this.timeService = checkNotNull(timeService); + this.batchMaxRowNums = bufferFlushMode.getBatchSize(); + this.batchIntervalMs = bufferFlushMode.getBatchIntervalMs(); + registerFlush(); + List fields = physicalDataType.getLogicalType().getChildren(); + final RowData.FieldGetter[] keyFieldGetters = + Arrays.stream(keyProjection) + .mapToObj( + targetField -> RowData.createFieldGetter( + fields.get(targetField), targetField)) + .toArray(RowData.FieldGetter[]::new); + this.keyExtractor = rowData -> createProjectedRow(rowData, RowKind.INSERT, keyFieldGetters); + this.valueCopyFunction = valueCopyFunction; + } + + @Override + public void write(RowData element, Context context) throws IOException, InterruptedException { + wrappedContext.setContext(context); + addToBuffer(element, context.timestamp()); + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + sinkBuffer(); + wrappedWriter.flush(endOfInput); + } + + @Override + public List snapshotState(long checkpointId) throws IOException { + return wrappedWriter.snapshotState(checkpointId); + } + + @Override + public void close() throws Exception { + if (!closed) { + closed = true; + wrappedWriter.close(); + } + } + + private void addToBuffer(RowData row, Long timestamp) throws IOException, InterruptedException { + RowData key = keyExtractor.apply(row); + RowData value = valueCopyFunction.apply(row); + reduceBuffer.put(key, new Tuple2<>(changeFlag(value), timestamp)); + + if (reduceBuffer.size() >= batchMaxRowNums) { + sinkBuffer(); + } + } + + private void registerFlush() { + if (closed) { + return; + } + timeService.registerTimer( + lastFlush + batchIntervalMs, + (t) -> { + if (t >= lastFlush + batchIntervalMs) { + sinkBuffer(); + } + registerFlush(); + }); + } + + private RowData changeFlag(RowData value) { + switch (value.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + value.setRowKind(UPDATE_AFTER); + break; + case UPDATE_BEFORE: + case DELETE: + value.setRowKind(DELETE); + } + return value; + } + + private void sinkBuffer() throws IOException, InterruptedException { + for (Tuple2 value : reduceBuffer.values()) { + wrappedContext.setTimestamp(value.f1); + wrappedWriter.write(value.f0, wrappedContext); + } + lastFlush = System.currentTimeMillis(); + reduceBuffer.clear(); + } + + @Override + public Collection prepareCommit() throws IOException, InterruptedException { + return wrappedWriter.prepareCommit(); + } + + /** + * Wrapper of {@link SinkWriter.Context}. + * + *

When records arrives, the {@link ReducingUpsertWriter} updates the current {@link + * SinkWriter.Context} and memorize the timestamp with the records. When flushing, the {@link + * ReducingUpsertWriter} will emit the records in the buffer with memorized timestamp. + */ + private static class WrappedContext implements SinkWriter.Context { + + private Long timestamp; + private SinkWriter.Context context; + + @Override + public long currentWatermark() { + checkNotNull(context, "context must be set before retrieving it."); + return context.currentWatermark(); + } + + @Override + public Long timestamp() { + return timestamp; + } + + public void setTimestamp(Long timestamp) { + this.timestamp = timestamp; + } + + public void setContext(SinkWriter.Context context) { + this.context = context; + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/SinkBufferFlushMode.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/SinkBufferFlushMode.java new file mode 100644 index 00000000000..c9b211c9858 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/SinkBufferFlushMode.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import java.io.Serializable; +import java.util.Objects; + +/** Sink buffer flush configuration. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +public class SinkBufferFlushMode implements Serializable { + + private static final int DISABLED_BATCH_SIZE = 0; + private static final long DISABLED_BATCH_INTERVAL = 0L; + + public static final SinkBufferFlushMode DISABLED = + new SinkBufferFlushMode(DISABLED_BATCH_SIZE, DISABLED_BATCH_INTERVAL); + + private final int batchSize; + private final long batchIntervalMs; + + public SinkBufferFlushMode(int batchSize, long batchIntervalMs) { + this.batchSize = batchSize; + this.batchIntervalMs = batchIntervalMs; + + // validation + if (isEnabled() + && !(batchSize > DISABLED_BATCH_SIZE + && batchIntervalMs > DISABLED_BATCH_INTERVAL)) { + throw new IllegalArgumentException( + String.format( + "batchSize and batchInterval must greater than zero if buffer flush is enabled," + + " but got batchSize=%s and batchIntervalMs=%s", + batchSize, batchIntervalMs)); + } + } + + public int getBatchSize() { + return batchSize; + } + + public long getBatchIntervalMs() { + return batchIntervalMs; + } + + public boolean isEnabled() { + return !(batchSize == DISABLED_BATCH_SIZE && batchIntervalMs == DISABLED_BATCH_INTERVAL); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SinkBufferFlushMode that = (SinkBufferFlushMode) o; + return batchSize == that.batchSize && batchIntervalMs == that.batchIntervalMs; + } + + @Override + public int hashCode() { + return Objects.hash(batchSize, batchIntervalMs); + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/UpsertKafkaDynamicTableFactory.java b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/UpsertKafkaDynamicTableFactory.java new file mode 100644 index 00000000000..9e59876f16f --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/UpsertKafkaDynamicTableFactory.java @@ -0,0 +1,437 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.inlong.sort.kafka.table; + +import org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.BoundedOptions; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.connectors.kafka.config.StartupMode; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.format.Format; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DeserializationFormatFactory; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.factories.SerializationFormatFactory; +import org.apache.flink.table.types.DataType; +import org.apache.flink.types.RowKind; + +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.DELIVERY_GUARANTEE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FIELDS_PREFIX; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.KEY_FORMAT; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.PROPS_BOOTSTRAP_SERVERS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_MODE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_SPECIFIC_OFFSETS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SCAN_BOUNDED_TIMESTAMP_MILLIS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_INTERVAL; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.SINK_PARALLELISM; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TOPIC; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.TRANSACTIONAL_ID_PREFIX; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FIELDS_INCLUDE; +import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.VALUE_FORMAT; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.PROPERTIES_PREFIX; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.autoCompleteSchemaRegistrySubject; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.createKeyFormatProjection; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.createValueFormatProjection; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getBoundedOptions; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getKafkaProperties; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getSourceTopicPattern; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.getSourceTopics; +import static org.apache.inlong.sort.kafka.table.KafkaConnectorOptionsUtil.validateScanBoundedMode; +import static org.apache.inlong.sort.protocol.constant.KafkaConstant.UPSERT_KAFKA; + +/** + * Upsert-Kafka factory. + * copied from org.apache.flink:flink-connector-kafka:3.2.0 + */ +public class UpsertKafkaDynamicTableFactory + implements + DynamicTableSourceFactory, + DynamicTableSinkFactory { + + public static final String IDENTIFIER = UPSERT_KAFKA; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + final Set> options = new HashSet<>(); + options.add(PROPS_BOOTSTRAP_SERVERS); + options.add(TOPIC); + options.add(KEY_FORMAT); + options.add(VALUE_FORMAT); + return options; + } + + @Override + public Set> optionalOptions() { + final Set> options = new HashSet<>(); + options.add(KEY_FIELDS_PREFIX); + options.add(VALUE_FIELDS_INCLUDE); + options.add(SINK_PARALLELISM); + options.add(SINK_BUFFER_FLUSH_INTERVAL); + options.add(SINK_BUFFER_FLUSH_MAX_ROWS); + options.add(SCAN_BOUNDED_MODE); + options.add(SCAN_BOUNDED_SPECIFIC_OFFSETS); + options.add(SCAN_BOUNDED_TIMESTAMP_MILLIS); + options.add(DELIVERY_GUARANTEE); + options.add(TRANSACTIONAL_ID_PREFIX); + return options; + } + + @Override + public Set> forwardOptions() { + return Stream.of(DELIVERY_GUARANTEE, TRANSACTIONAL_ID_PREFIX).collect(Collectors.toSet()); + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); + + ReadableConfig tableOptions = helper.getOptions(); + DecodingFormat> keyDecodingFormat = + helper.discoverDecodingFormat(DeserializationFormatFactory.class, KEY_FORMAT); + DecodingFormat> valueDecodingFormat = + helper.discoverDecodingFormat(DeserializationFormatFactory.class, VALUE_FORMAT); + + // Validate the option data type. + helper.validateExcept(PROPERTIES_PREFIX); + validateSource( + tableOptions, + keyDecodingFormat, + valueDecodingFormat, + context.getPrimaryKeyIndexes()); + + Tuple2 keyValueProjections = + createKeyValueProjections(context.getCatalogTable()); + String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + Properties properties = getKafkaProperties(context.getCatalogTable().getOptions()); + // always use earliest to keep data integrity + StartupMode earliest = StartupMode.EARLIEST; + + final BoundedOptions boundedOptions = getBoundedOptions(tableOptions); + + return new KafkaDynamicSource( + context.getPhysicalRowDataType(), + keyDecodingFormat, + new DecodingFormatWrapper(valueDecodingFormat), + keyValueProjections.f0, + keyValueProjections.f1, + keyPrefix, + getSourceTopics(tableOptions), + getSourceTopicPattern(tableOptions), + properties, + earliest, + Collections.emptyMap(), + 0, + boundedOptions.boundedMode, + boundedOptions.specificOffsets, + boundedOptions.boundedTimestampMillis, + true, + context.getObjectIdentifier().asSummaryString()); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper( + this, autoCompleteSchemaRegistrySubject(context)); + + final ReadableConfig tableOptions = helper.getOptions(); + + EncodingFormat> keyEncodingFormat = + helper.discoverEncodingFormat(SerializationFormatFactory.class, KEY_FORMAT); + EncodingFormat> valueEncodingFormat = + helper.discoverEncodingFormat(SerializationFormatFactory.class, VALUE_FORMAT); + + // Validate the option data type. + helper.validateExcept(PROPERTIES_PREFIX); + validateSink( + tableOptions, + keyEncodingFormat, + valueEncodingFormat, + context.getPrimaryKeyIndexes()); + KafkaConnectorOptionsUtil.validateDeliveryGuarantee(tableOptions); + + Tuple2 keyValueProjections = + createKeyValueProjections(context.getCatalogTable()); + final String keyPrefix = tableOptions.getOptional(KEY_FIELDS_PREFIX).orElse(null); + final Properties properties = getKafkaProperties(context.getCatalogTable().getOptions()); + + Integer parallelism = tableOptions.get(SINK_PARALLELISM); + + int batchSize = tableOptions.get(SINK_BUFFER_FLUSH_MAX_ROWS); + Duration batchInterval = tableOptions.get(SINK_BUFFER_FLUSH_INTERVAL); + SinkBufferFlushMode flushMode = + new SinkBufferFlushMode(batchSize, batchInterval.toMillis()); + + // use {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner}. + // it will use hash partition if key is set else in round-robin behaviour. + return new KafkaDynamicSink( + context.getPhysicalRowDataType(), + context.getPhysicalRowDataType(), + keyEncodingFormat, + new EncodingFormatWrapper(valueEncodingFormat), + keyValueProjections.f0, + keyValueProjections.f1, + keyPrefix, + tableOptions.get(TOPIC).get(0), + properties, + null, + tableOptions.get(DELIVERY_GUARANTEE), + true, + flushMode, + parallelism, + tableOptions.get(TRANSACTIONAL_ID_PREFIX)); + } + + private Tuple2 createKeyValueProjections(ResolvedCatalogTable catalogTable) { + ResolvedSchema schema = catalogTable.getResolvedSchema(); + // primary key should validated earlier + List keyFields = schema.getPrimaryKey().get().getColumns(); + DataType physicalDataType = schema.toPhysicalRowDataType(); + + Configuration tableOptions = Configuration.fromMap(catalogTable.getOptions()); + // upsert-kafka will set key.fields to primary key fields by default + tableOptions.set(KEY_FIELDS, keyFields); + + int[] keyProjection = createKeyFormatProjection(tableOptions, physicalDataType); + int[] valueProjection = createValueFormatProjection(tableOptions, physicalDataType); + + return Tuple2.of(keyProjection, valueProjection); + } + + // -------------------------------------------------------------------------------------------- + // Validation + // -------------------------------------------------------------------------------------------- + + private static void validateSource( + ReadableConfig tableOptions, + Format keyFormat, + Format valueFormat, + int[] primaryKeyIndexes) { + validateTopic(tableOptions); + validateScanBoundedMode(tableOptions); + validateFormat(keyFormat, valueFormat, tableOptions); + validatePKConstraints(primaryKeyIndexes); + } + + private static void validateSink( + ReadableConfig tableOptions, + Format keyFormat, + Format valueFormat, + int[] primaryKeyIndexes) { + validateTopic(tableOptions); + validateFormat(keyFormat, valueFormat, tableOptions); + validatePKConstraints(primaryKeyIndexes); + validateSinkBufferFlush(tableOptions); + } + + private static void validateTopic(ReadableConfig tableOptions) { + List topic = tableOptions.get(TOPIC); + if (topic.size() > 1) { + throw new ValidationException( + "The 'upsert-kafka' connector doesn't support topic list now. " + + "Please use single topic as the value of the parameter 'topic'."); + } + } + + private static void validateFormat( + Format keyFormat, Format valueFormat, ReadableConfig tableOptions) { + if (!keyFormat.getChangelogMode().containsOnly(RowKind.INSERT)) { + String identifier = tableOptions.get(KEY_FORMAT); + throw new ValidationException( + String.format( + "'upsert-kafka' connector doesn't support '%s' as key format, " + + "because '%s' is not in insert-only mode.", + identifier, identifier)); + } + if (!valueFormat.getChangelogMode().containsOnly(RowKind.INSERT)) { + String identifier = tableOptions.get(VALUE_FORMAT); + throw new ValidationException( + String.format( + "'upsert-kafka' connector doesn't support '%s' as value format, " + + "because '%s' is not in insert-only mode.", + identifier, identifier)); + } + } + + private static void validatePKConstraints(int[] schema) { + if (schema.length == 0) { + throw new ValidationException( + "'upsert-kafka' tables require to define a PRIMARY KEY constraint. " + + "The PRIMARY KEY specifies which columns should be read from or write to the Kafka message key. " + + "The PRIMARY KEY also defines records in the 'upsert-kafka' table should update or delete on which keys."); + } + } + + private static void validateSinkBufferFlush(ReadableConfig tableOptions) { + int flushMaxRows = tableOptions.get(SINK_BUFFER_FLUSH_MAX_ROWS); + long flushIntervalMs = tableOptions.get(SINK_BUFFER_FLUSH_INTERVAL).toMillis(); + if (flushMaxRows > 0 && flushIntervalMs > 0) { + // flush is enabled + return; + } + if (flushMaxRows <= 0 && flushIntervalMs <= 0) { + // flush is disabled + return; + } + // one of them is set which is not allowed + throw new ValidationException( + String.format( + "'%s' and '%s' must be set to be greater than zero together to enable sink buffer flushing.", + SINK_BUFFER_FLUSH_MAX_ROWS.key(), SINK_BUFFER_FLUSH_INTERVAL.key())); + } + + // -------------------------------------------------------------------------------------------- + // Format wrapper + // -------------------------------------------------------------------------------------------- + + /** + * It is used to wrap the decoding format and expose the desired changelog mode. It's only works + * for insert-only format. + */ + protected static class DecodingFormatWrapper + implements + DecodingFormat> { + + private final DecodingFormat> innerDecodingFormat; + + private static final ChangelogMode SOURCE_CHANGELOG_MODE = + ChangelogMode.newBuilder() + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + + public DecodingFormatWrapper( + DecodingFormat> innerDecodingFormat) { + this.innerDecodingFormat = innerDecodingFormat; + } + + @Override + public DeserializationSchema createRuntimeDecoder( + DynamicTableSource.Context context, DataType producedDataType) { + return innerDecodingFormat.createRuntimeDecoder(context, producedDataType); + } + + @Override + public ChangelogMode getChangelogMode() { + return SOURCE_CHANGELOG_MODE; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + DecodingFormatWrapper that = (DecodingFormatWrapper) obj; + return Objects.equals(innerDecodingFormat, that.innerDecodingFormat); + } + + @Override + public int hashCode() { + return Objects.hash(innerDecodingFormat); + } + } + + /** + * It is used to wrap the encoding format and expose the desired changelog mode. It's only works + * for insert-only format. + */ + protected static class EncodingFormatWrapper + implements + EncodingFormat> { + + private final EncodingFormat> innerEncodingFormat; + + public static final ChangelogMode SINK_CHANGELOG_MODE = + ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.UPDATE_AFTER) + .addContainedKind(RowKind.DELETE) + .build(); + + public EncodingFormatWrapper( + EncodingFormat> innerEncodingFormat) { + this.innerEncodingFormat = innerEncodingFormat; + } + + @Override + public SerializationSchema createRuntimeEncoder( + DynamicTableSink.Context context, DataType consumedDataType) { + return innerEncodingFormat.createRuntimeEncoder(context, consumedDataType); + } + + @Override + public ChangelogMode getChangelogMode() { + return SINK_CHANGELOG_MODE; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + EncodingFormatWrapper that = (EncodingFormatWrapper) obj; + return Objects.equals(innerEncodingFormat, that.innerEncodingFormat); + } + + @Override + public int hashCode() { + return Objects.hash(innerEncodingFormat); + } + } +} diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 00000000000..5f3c21a9d46 --- /dev/null +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,17 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.inlong.sort.kafka.table.KafkaDynamicTableFactory +org.apache.inlong.sort.kafka.table.UpsertKafkaDynamicTableFactory diff --git a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/pom.xml b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/pom.xml index ce153aecea9..1cb1e7f0bae 100644 --- a/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/pom.xml +++ b/inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/pom.xml @@ -36,6 +36,7 @@ elasticsearch-base elasticsearch6 elasticsearch7 + kafka diff --git a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/client/consumer_impl.go b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/client/consumer_impl.go index 6cc0174835f..c78f2469b5d 100644 --- a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/client/consumer_impl.go +++ b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/client/consumer_impl.go @@ -68,7 +68,8 @@ type consumer struct { masterHBRetry int heartbeatManager *heartbeatManager unreportedTimes int - done chan struct{} + cancel context.CancelFunc + routineClosed chan struct{} closeOnce sync.Once } @@ -105,72 +106,92 @@ func NewConsumer(config *config.Config) (Consumer, error) { client: client, visitToken: util.InvalidValue, unreportedTimes: 0, - done: make(chan struct{}), + routineClosed: make(chan struct{}), } + + ctx := context.Background() + ctx, c.cancel = context.WithCancel(ctx) + c.subInfo.SetClientID(clientID) hbm := newHBManager(c) c.heartbeatManager = hbm - err = c.register2Master(true) - if err != nil { - return nil, err - } - c.heartbeatManager.registerMaster(c.master.Address) - go c.processRebalanceEvent() + + go c.routine(ctx) + go c.processRebalanceEvent(ctx) + log.Infof("[CONSUMER] start consumer success, client=%s", clientID) return c, nil } -func (c *consumer) register2Master(needChange bool) error { - if needChange { - c.selector.Refresh(c.config.Consumer.Masters) +func (c *consumer) routine(ctx context.Context) { + defer close(c.routineClosed) + + for { + select { + case <-ctx.Done(): + return + default: + } + // select master node, err := c.selector.Select(c.config.Consumer.Masters) if err != nil { - return err + log.Errorf("[CONSUMER] select error %v", err) + time.Sleep(time.Second) + continue } c.master = node - } - retryCount := 0 - for { - rsp, err := c.sendRegRequest2Master() - if err != nil || !rsp.GetSuccess() { - if err != nil { - log.Errorf("[CONSUMER]register2Master error %s", err.Error()) - } else if rsp.GetErrCode() == errs.RetConsumeGroupForbidden || - rsp.GetErrCode() == errs.RetConsumeContentForbidden { - log.Warnf("[CONSUMER] register2master(%s) failure exist register, client=%s, error: %s", - c.master.Address, c.clientID, rsp.GetErrMsg()) - return errs.New(rsp.GetErrCode(), rsp.GetErrMsg()) + log.Infof("[CONSUMER] master %+v", c.master) + // register to master + if err := c.register2Master(ctx); err != nil { + log.Errorf("[CONSUMER] register2Master error %v", err) + time.Sleep(time.Second) + continue + } + c.lastMasterHb = time.Now().UnixMilli() + // heartbeat to master + time.Sleep(c.config.Heartbeat.Interval / 2) + if err := c.heartbeat2Master(ctx); err != nil { + log.Errorf("[CONSUMER] heartbeat2Master error %v", err) + } else { + c.lastMasterHb = time.Now().UnixMilli() + } + heartbeatRetry := 0 + for { + time.Sleep(c.config.Heartbeat.Interval) + select { + case <-ctx.Done(): + return + default: } - - if !c.master.HasNext { - if err != nil { - return err - } - if rsp != nil { - log.Errorf("[CONSUMER] register2master(%s) failure exist register, client=%s, error: %s", - c.master.Address, c.clientID, rsp.GetErrMsg()) - } + if heartbeatRetry >= c.config.Heartbeat.MaxRetryTimes { break } - retryCount++ - log.Warnf("[CONSUMER] register2master(%s) failure, client=%s, retry count=%d", - c.master.Address, c.clientID, retryCount) - if c.master, err = c.selector.Select(c.config.Consumer.Masters); err != nil { - return err + if err := c.heartbeat2Master(ctx); err != nil { + log.Errorf("[CONSUMER] heartbeat2Master error %v", err) + heartbeatRetry++ + continue + } else { + heartbeatRetry = 0 + c.lastMasterHb = time.Now().UnixMilli() } - continue } - log.Infof("register2Master response %s", rsp.String()) + } +} - c.masterHBRetry = 0 - c.processRegisterResponseM2C(rsp) - break +func (c *consumer) register2Master(ctx context.Context) error { + rsp, err := c.sendRegRequest2Master(ctx) + if err != nil { + return err + } + if !rsp.GetSuccess() { + return errs.New(rsp.GetErrCode(), rsp.GetErrMsg()) } + c.processRegisterResponseM2C(rsp) return nil } -func (c *consumer) sendRegRequest2Master() (*protocol.RegisterResponseM2C, error) { - ctx, cancel := context.WithTimeout(context.Background(), c.config.Net.ReadTimeout) +func (c *consumer) sendRegRequest2Master(ctx context.Context) (*protocol.RegisterResponseM2C, error) { + ctx, cancel := context.WithTimeout(ctx, c.config.Net.ReadTimeout) defer cancel() m := &metadata.Metadata{} @@ -207,7 +228,6 @@ func (c *consumer) processRegisterResponseM2C(rsp *protocol.RegisterResponseM2C) if rsp.GetAuthorizedInfo() != nil { c.processAuthorizedToken(rsp.GetAuthorizedInfo()) } - c.lastMasterHb = time.Now().UnixNano() / int64(time.Millisecond) } func (c *consumer) processAuthorizedToken(info *protocol.MasterAuthorizedInfo) { @@ -215,12 +235,84 @@ func (c *consumer) processAuthorizedToken(info *protocol.MasterAuthorizedInfo) { c.authorizedInfo = info.GetAuthAuthorizedToken() } -// GetMessage implementation of TubeMQ consumer. -func (c *consumer) GetMessage() (*ConsumerResult, error) { - err := c.checkPartitionErr() +func (c *consumer) heartbeat2Master(ctx context.Context) error { + rsp, err := c.sendHeartbeat2Master(ctx) if err != nil { - return nil, err + return err + } + if !rsp.GetSuccess() { + return errs.New(rsp.GetErrCode(), rsp.GetErrMsg()) } + c.processHBResponseM2C(rsp) + return nil +} + +func (c *consumer) sendHeartbeat2Master(ctx context.Context) (*protocol.HeartResponseM2C, error) { + if time.Now().UnixNano()/int64(time.Millisecond)-c.lastMasterHb > 30000 { + c.rmtDataCache.HandleExpiredPartitions(c.config.Consumer.MaxConfirmWait) + } + m := &metadata.Metadata{} + node := &metadata.Node{} + node.SetHost(util.GetLocalHost()) + node.SetAddress(c.master.Address) + m.SetNode(node) + sub := &metadata.SubscribeInfo{} + sub.SetGroup(c.config.Consumer.Group) + m.SetSubscribeInfo(sub) + auth := &protocol.AuthenticateInfo{} + if c.needGenMasterCertificateInfo(true) { + util.GenMasterAuthenticateToken(auth, c.config.Net.Auth.UserName, c.config.Net.Auth.Password) + } + c.unreportedTimes++ + if c.unreportedTimes > c.config.Consumer.MaxSubInfoReportInterval { + m.SetReportTimes(true) + c.unreportedTimes = 0 + } + + ctx, cancel := context.WithTimeout(ctx, c.config.Net.ReadTimeout) + defer cancel() + rsp, err := c.client.HeartRequestC2M(ctx, m, c.subInfo, c.rmtDataCache) + return rsp, err +} + +func (c *consumer) processHBResponseM2C(rsp *protocol.HeartResponseM2C) { + c.masterHBRetry = 0 + if !rsp.GetNotAllocated() { + c.subInfo.CASIsNotAllocated(1, 0) + } + if rsp.GetDefFlowCheckId() != 0 || rsp.GetGroupFlowCheckId() != 0 { + if rsp.GetDefFlowCheckId() != 0 { + c.rmtDataCache.UpdateDefFlowCtrlInfo(rsp.GetDefFlowCheckId(), rsp.GetDefFlowControlInfo()) + } + qryPriorityID := c.rmtDataCache.GetQryPriorityID() + if rsp.GetQryPriorityId() != 0 { + qryPriorityID = rsp.GetQryPriorityId() + } + c.rmtDataCache.UpdateGroupFlowCtrlInfo(qryPriorityID, rsp.GetGroupFlowCheckId(), rsp.GetGroupFlowControlInfo()) + } + if rsp.GetAuthorizedInfo() != nil { + c.processAuthorizedToken(rsp.GetAuthorizedInfo()) + } + if rsp.GetRequireAuth() { + atomic.StoreInt32(&c.nextAuth2Master, 1) + } + if rsp.GetEvent() != nil { + event := rsp.GetEvent() + subscribeInfo := make([]*metadata.SubscribeInfo, 0, len(event.GetSubscribeInfo())) + for _, sub := range event.GetSubscribeInfo() { + s, err := metadata.NewSubscribeInfo(sub) + if err != nil { + continue + } + subscribeInfo = append(subscribeInfo, s) + } + e := metadata.NewEvent(event.GetRebalanceId(), event.GetOpType(), subscribeInfo) + c.rmtDataCache.OfferEvent(e) + } +} + +// GetMessage implementation of TubeMQ consumer. +func (c *consumer) GetMessage() (*ConsumerResult, error) { partition, bookedTime, err := c.rmtDataCache.SelectPartition() if err != nil { return nil, err @@ -372,35 +464,36 @@ func (c *consumer) GetClientID() string { func (c *consumer) Close() { c.closeOnce.Do(func() { log.Infof("[CONSUMER]Begin to close consumer, client=%s", c.clientID) - close(c.done) + c.cancel() c.heartbeatManager.close() c.close2Master() c.closeAllBrokers() c.client.Close() + <-c.routineClosed log.Infof("[CONSUMER]Consumer has been closed successfully, client=%s", c.clientID) }) } -func (c *consumer) processRebalanceEvent() { +func (c *consumer) processRebalanceEvent(ctx context.Context) { log.Info("[CONSUMER]Rebalance event Handler starts!") for { select { + case <-ctx.Done(): + log.Info("[CONSUMER] Rebalance event Handler stopped!") + return case event, ok := <-c.rmtDataCache.EventCh: if ok { + log.Infof("%+v", event) c.rmtDataCache.ClearEvent() switch event.GetEventType() { case metadata.Disconnect, metadata.OnlyDisconnect: c.disconnect2Broker(event) c.rmtDataCache.OfferEventResult(event) case metadata.Connect, metadata.OnlyConnect: - c.connect2Broker(event) + c.connect2Broker(ctx, event) c.rmtDataCache.OfferEventResult(event) } } - case <-c.done: - log.Infof("[CONSUMER]Rebalance done, client=%s", c.clientID) - log.Info("[CONSUMER] Rebalance event Handler stopped!") - return } } } @@ -424,16 +517,26 @@ func (c *consumer) unregister2Broker(unRegPartitions map[*metadata.Node][]*metad if len(unRegPartitions) == 0 { return } - + var wg sync.WaitGroup for _, partitions := range unRegPartitions { for _, partition := range partitions { log.Tracef("unregister2Brokers, partition key=%s", partition.GetPartitionKey()) - c.sendUnregisterReq2Broker(partition) + partition := partition + wg.Add(1) + go func() { + defer wg.Done() + if err := c.sendUnregisterReq2Broker(partition); err != nil { + log.Errorf("[CONSUMER] unregister partition %+v failed, error %v", partition, err) + } else { + log.Infof("[connect2Broker] unregister partition %+v success", partition) + } + }() } } + wg.Wait() } -func (c *consumer) sendUnregisterReq2Broker(partition *metadata.Partition) { +func (c *consumer) sendUnregisterReq2Broker(partition *metadata.Partition) error { ctx, cancel := context.WithTimeout(context.Background(), c.config.Net.ReadTimeout) defer cancel() @@ -454,20 +557,29 @@ func (c *consumer) sendUnregisterReq2Broker(partition *metadata.Partition) { rsp, err := c.client.UnregisterRequestC2B(ctx, m, c.subInfo) if err != nil { log.Errorf("[CONSUMER] fail to unregister partition %s, error %s", partition, err.Error()) - return + return err } if !rsp.GetSuccess() { log.Errorf("[CONSUMER] fail to unregister partition %s, err code: %d, error msg %s", partition, rsp.GetErrCode(), rsp.GetErrMsg()) + return errs.New(rsp.GetErrCode(), rsp.GetErrMsg()) } + return nil } -func (c *consumer) connect2Broker(event *metadata.ConsumerEvent) { +func (c *consumer) connect2Broker(ctx context.Context, event *metadata.ConsumerEvent) { log.Tracef("[connect2Broker] connect event begin, client=%s", c.clientID) if len(event.GetSubscribeInfo()) > 0 { unsubPartitions := c.rmtDataCache.FilterPartitions(event.GetSubscribeInfo()) + n := len(unsubPartitions) if len(unsubPartitions) > 0 { - for _, partition := range unsubPartitions { + for i, partition := range unsubPartitions { + select { + case <-ctx.Done(): + return + default: + } + node := &metadata.Node{} node.SetHost(util.GetLocalHost()) node.SetAddress(partition.GetBroker().GetAddress()) @@ -482,6 +594,7 @@ func (c *consumer) connect2Broker(event *metadata.ConsumerEvent) { return } + log.Infof("[connect2Broker] %v/%v register partition %+v success", i, n, partition) c.rmtDataCache.AddNewPartition(partition) c.heartbeatManager.registerBroker(partition.GetBroker()) } diff --git a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/client/heartbeat.go b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/client/heartbeat.go index df6d0794459..67e4faa134e 100644 --- a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/client/heartbeat.go +++ b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/client/heartbeat.go @@ -21,7 +21,6 @@ import ( "context" "strings" "sync" - "sync/atomic" "time" "github.com/apache/inlong/inlong-tubemq/tubemq-client-twins/tubemq-client-go/errs" @@ -61,9 +60,6 @@ func (h *heartbeatManager) registerMaster(address string) { if h.producer != nil { heartbeatInterval = h.producer.config.Heartbeat.Interval / 2 heartbeatFunc = h.producerHB2Master - } else if h.consumer != nil { - heartbeatInterval = h.consumer.config.Heartbeat.Interval / 2 - heartbeatFunc = h.consumerHB2Master } if !ok { @@ -133,58 +129,6 @@ func (h *heartbeatManager) producerHB2Master() { h.resetMasterHeartbeat() } -func (h *heartbeatManager) consumerHB2Master() { - if time.Now().UnixNano()/int64(time.Millisecond)-h.consumer.lastMasterHb > 30000 { - h.consumer.rmtDataCache.HandleExpiredPartitions(h.consumer.config.Consumer.MaxConfirmWait) - } - m := &metadata.Metadata{} - node := &metadata.Node{} - node.SetHost(util.GetLocalHost()) - node.SetAddress(h.consumer.master.Address) - m.SetNode(node) - sub := &metadata.SubscribeInfo{} - sub.SetGroup(h.consumer.config.Consumer.Group) - m.SetSubscribeInfo(sub) - auth := &protocol.AuthenticateInfo{} - if h.consumer.needGenMasterCertificateInfo(true) { - util.GenMasterAuthenticateToken(auth, h.consumer.config.Net.Auth.UserName, h.consumer.config.Net.Auth.Password) - } - h.consumer.unreportedTimes++ - if h.consumer.unreportedTimes > h.consumer.config.Consumer.MaxSubInfoReportInterval { - m.SetReportTimes(true) - h.consumer.unreportedTimes = 0 - } - - rsp, err := h.sendHeartbeatC2M(m) - if err == nil { - h.consumer.masterHBRetry = 0 - h.processHBResponseM2C(rsp) - h.resetMasterHeartbeat() - return - } - h.consumer.masterHBRetry++ - h.resetMasterHeartbeat() - hbNoNode := rsp != nil && rsp.GetErrCode() == errs.RetErrHBNoNode - standByException := false - if e, ok := err.(*errs.Error); ok { - standByException = strings.Index(e.Msg, "StandbyException") != -1 - } - if (h.consumer.masterHBRetry >= h.consumer.config.Heartbeat.MaxRetryTimes) || standByException || hbNoNode { - h.deleteHeartbeat(h.consumer.master.Address) - go func() { - err := h.consumer.register2Master(!hbNoNode) - if err != nil { - log.Warnf("[CONSUMER] heartBeat2Master failure to (%s) : %s, client=%s", - h.consumer.master.Address, rsp.GetErrMsg(), h.consumer.clientID) - return - } - h.registerMaster(h.consumer.master.Address) - log.Infof("[CONSUMER] heartBeat2Master success to (%s), client=%s", - h.consumer.master.Address, h.consumer.clientID) - }() - } -} - func (h *heartbeatManager) resetMasterHeartbeat() { h.mu.Lock() defer h.mu.Unlock() @@ -224,42 +168,6 @@ func (h *heartbeatManager) processHBResponseM2P(rsp *protocol.HeartResponseM2P) h.producer.updateTopicConfigure(topicInfos) } -func (h *heartbeatManager) processHBResponseM2C(rsp *protocol.HeartResponseM2C) { - h.consumer.masterHBRetry = 0 - if !rsp.GetNotAllocated() { - h.consumer.subInfo.CASIsNotAllocated(1, 0) - } - if rsp.GetDefFlowCheckId() != 0 || rsp.GetGroupFlowCheckId() != 0 { - if rsp.GetDefFlowCheckId() != 0 { - h.consumer.rmtDataCache.UpdateDefFlowCtrlInfo(rsp.GetDefFlowCheckId(), rsp.GetDefFlowControlInfo()) - } - qryPriorityID := h.consumer.rmtDataCache.GetQryPriorityID() - if rsp.GetQryPriorityId() != 0 { - qryPriorityID = rsp.GetQryPriorityId() - } - h.consumer.rmtDataCache.UpdateGroupFlowCtrlInfo(qryPriorityID, rsp.GetGroupFlowCheckId(), rsp.GetGroupFlowControlInfo()) - } - if rsp.GetAuthorizedInfo() != nil { - h.consumer.processAuthorizedToken(rsp.GetAuthorizedInfo()) - } - if rsp.GetRequireAuth() { - atomic.StoreInt32(&h.consumer.nextAuth2Master, 1) - } - if rsp.GetEvent() != nil { - event := rsp.GetEvent() - subscribeInfo := make([]*metadata.SubscribeInfo, 0, len(event.GetSubscribeInfo())) - for _, sub := range event.GetSubscribeInfo() { - s, err := metadata.NewSubscribeInfo(sub) - if err != nil { - continue - } - subscribeInfo = append(subscribeInfo, s) - } - e := metadata.NewEvent(event.GetRebalanceId(), event.GetOpType(), subscribeInfo) - h.consumer.rmtDataCache.OfferEventAndNotify(e) - } -} - func (h *heartbeatManager) nextHeartbeatInterval() time.Duration { var interval time.Duration if h.producer != nil { @@ -350,10 +258,6 @@ func (h *heartbeatManager) close() { defer h.mu.Unlock() for _, heartbeat := range h.heartbeats { - if !heartbeat.timer.Stop() { - <-heartbeat.timer.C - } - heartbeat.timer = nil + heartbeat.timer.Stop() } - h.heartbeats = nil } diff --git a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/remote/remote.go b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/remote/remote.go index 13b7a15472d..0563c323051 100644 --- a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/remote/remote.go +++ b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/remote/remote.go @@ -185,21 +185,20 @@ func (r *RmtDataCache) UpdateGroupFlowCtrlInfo(qryPriorityID int32, flowCtrlID i } } -// OfferEventAndNotify offers a consumer event and notifies the consumer method and notify the consumer to consume. -func (r *RmtDataCache) OfferEventAndNotify(event *metadata.ConsumerEvent) { - r.eventReadMu.Lock() - defer r.eventReadMu.Unlock() - r.rebalanceResults = append(r.rebalanceResults, event) - e := r.rebalanceResults[0] - r.rebalanceResults = r.rebalanceResults[1:] - r.EventCh <- e +// OfferEvent offers a consumer event and notifies the consumer method and notify the consumer to consume. +func (r *RmtDataCache) OfferEvent(event *metadata.ConsumerEvent) { + r.EventCh <- event } // ClearEvent clears all the events. func (r *RmtDataCache) ClearEvent() { - r.eventWriteMu.Lock() - defer r.eventWriteMu.Unlock() - r.rebalanceResults = r.rebalanceResults[:0] + for { + select { + case <-r.EventCh: + default: + return + } + } } // OfferEventResult offers a consumer event. diff --git a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/rpc/master.go b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/rpc/master.go index 386f619c9e9..38274c14602 100644 --- a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/rpc/master.go +++ b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/rpc/master.go @@ -24,6 +24,7 @@ import ( "github.com/apache/inlong/inlong-tubemq/tubemq-client-twins/tubemq-client-go/codec" "github.com/apache/inlong/inlong-tubemq/tubemq-client-twins/tubemq-client-go/errs" + "github.com/apache/inlong/inlong-tubemq/tubemq-client-twins/tubemq-client-go/log" "github.com/apache/inlong/inlong-tubemq/tubemq-client-twins/tubemq-client-go/metadata" "github.com/apache/inlong/inlong-tubemq/tubemq-client-twins/tubemq-client-go/protocol" "github.com/apache/inlong/inlong-tubemq/tubemq-client-twins/tubemq-client-go/remote" @@ -199,6 +200,7 @@ func (c *rpcClient) HeartRequestC2M(ctx context.Context, metadata *metadata.Meta } } if event != nil { + log.Infof("report Event: %v", event) ep := &protocol.EventProto{ RebalanceId: proto.Int64(event.GetRebalanceID()), OpType: proto.Int32(event.GetEventType()), diff --git a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/selector/ip_selector.go b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/selector/ip_selector.go index b1193645ec7..a87075e3fb1 100644 --- a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/selector/ip_selector.go +++ b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/selector/ip_selector.go @@ -19,6 +19,8 @@ package selector import ( "errors" + "fmt" + "net" "strings" "sync" ) @@ -57,11 +59,21 @@ func (s *ipSelector) Select(serviceName string) (*Node, error) { } s.services[serviceName] = services } + address := services.addresses[services.nextIndex] + services.nextIndex = (services.nextIndex + 1) % len(services.addresses) + host, port, err := net.SplitHostPort(address) + if err != nil { + return nil, err + } + ips, err := net.LookupHost(host) + if err != nil { + return nil, err + } + address = fmt.Sprintf("%v:%v", ips[0], port) node := &Node{ ServiceName: serviceName, - Address: services.addresses[services.nextIndex], + Address: address, } - services.nextIndex = (services.nextIndex + 1) % len(services.addresses) if services.nextIndex > 0 { node.HasNext = true } diff --git a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/selector/ip_selector_test.go b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/selector/ip_selector_test.go index c091262f82f..1303b6bfd56 100644 --- a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/selector/ip_selector_test.go +++ b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/selector/ip_selector_test.go @@ -34,17 +34,6 @@ func TestSingleIP(t *testing.T) { assert.Equal(t, node.ServiceName, "127.0.0.1:9092") } -func TestSingleDNS(t *testing.T) { - serviceName := "tubemq:8081" - selector, err := Get("dns") - assert.Nil(t, err) - node, err := selector.Select(serviceName) - assert.Nil(t, err) - assert.Equal(t, node.HasNext, false) - assert.Equal(t, node.Address, "tubemq:8081") - assert.Equal(t, node.ServiceName, "tubemq:8081") -} - func TestMultipleIP(t *testing.T) { serviceName := "127.0.0.1:9091,127.0.0.1:9092,127.0.0.1:9093,127.0.0.1:9094" selector, err := Get("dns") @@ -71,32 +60,6 @@ func TestMultipleIP(t *testing.T) { assert.Equal(t, "127.0.0.1:9091,127.0.0.1:9092,127.0.0.1:9093,127.0.0.1:9094", node.ServiceName) } -func TestMultipleDNS(t *testing.T) { - serviceName := "tubemq:8081,tubemq:8082,tubemq:8083,tubemq:8084" - selector, err := Get("dns") - assert.Nil(t, err) - node, err := selector.Select(serviceName) - assert.Nil(t, err) - assert.Equal(t, true, node.HasNext) - assert.Equal(t, "tubemq:8081", node.Address) - assert.Equal(t, "tubemq:8081,tubemq:8082,tubemq:8083,tubemq:8084", node.ServiceName) - - node, err = selector.Select(serviceName) - assert.Equal(t, true, node.HasNext) - assert.Equal(t, "tubemq:8082", node.Address) - assert.Equal(t, "tubemq:8081,tubemq:8082,tubemq:8083,tubemq:8084", node.ServiceName) - - node, err = selector.Select(serviceName) - assert.Equal(t, true, node.HasNext) - assert.Equal(t, "tubemq:8083", node.Address) - assert.Equal(t, "tubemq:8081,tubemq:8082,tubemq:8083,tubemq:8084", node.ServiceName) - - node, err = selector.Select(serviceName) - assert.Equal(t, false, node.HasNext) - assert.Equal(t, "tubemq:8084", node.Address) - assert.Equal(t, "tubemq:8081,tubemq:8082,tubemq:8083,tubemq:8084", node.ServiceName) -} - func TestEmptyService(t *testing.T) { serviceName := "" selector, err := Get("ip") diff --git a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/sub/info.go b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/sub/info.go index 1989024f762..fb7efc1069f 100644 --- a/inlong-tubemq/tubemq-client-twins/tubemq-client-go/sub/info.go +++ b/inlong-tubemq/tubemq-client-twins/tubemq-client-go/sub/info.go @@ -20,6 +20,7 @@ package sub import ( "strconv" + "strings" "sync/atomic" "time" @@ -64,16 +65,10 @@ func NewSubInfo(config *config.Config) *SubInfo { s.topicFilter = make(map[string]bool) for topic, filters := range config.Consumer.TopicFilters { cond := topic + "#" - count := 0 if len(filters) > 0 { s.topicFilter[topic] = true } - for _, filter := range filters { - if count > 0 { - cond += "," - } - cond += filter - } + cond += strings.Join(filters, ",") s.topicConds = append(s.topicConds, cond) } if config.Consumer.BoundConsume { diff --git a/licenses/inlong-agent/LICENSE b/licenses/inlong-agent/LICENSE index 33a51c916aa..cd2f5290726 100644 --- a/licenses/inlong-agent/LICENSE +++ b/licenses/inlong-agent/LICENSE @@ -541,6 +541,7 @@ MIT licenses The following components are provided under MIT license. See project link for details. The text of each license is also included at licenses/LICENSE-[project].txt. + com.qcloud:cos_api:jar:5.6.54 - cos-java-sdk (https://github.com/tencentyun/cos-java-sdk-v5), (MIT License) net.sourceforge.argparse4j:argparse4j:0.7.0 - argparse4j (https://github.com/argparse4j/argparse4j/tree/argparse4j-0.7.0), (MIT) org.bouncycastle:bcpkix-jdk15on:1.69 - Bouncy Castle PKIX, CMS, EAC, TSP, PKCS, OCSP, CMP, and CRMF APIs (https://www.bouncycastle.org/java.html), (MIT License) org.bouncycastle:bcprov-ext-jdk15on:1.69 - Bouncy Castle Provider (https://www.bouncycastle.org/java.html), (MIT License) diff --git a/licenses/inlong-agent/licenses/LICENSE-cos-java-sdk.txt b/licenses/inlong-agent/licenses/LICENSE-cos-java-sdk.txt new file mode 100644 index 00000000000..4b32321f985 --- /dev/null +++ b/licenses/inlong-agent/licenses/LICENSE-cos-java-sdk.txt @@ -0,0 +1,21 @@ +MIT License + +Copyright (c) 2017 腾讯云 + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. \ No newline at end of file diff --git a/licenses/inlong-sort-connectors/LICENSE b/licenses/inlong-sort-connectors/LICENSE index 5539a8ce8b1..736c7c4ccd9 100644 --- a/licenses/inlong-sort-connectors/LICENSE +++ b/licenses/inlong-sort-connectors/LICENSE @@ -999,6 +999,23 @@ License : https://github.com/apache/flink/blob/master/LICENSE Source : org.apache.flink:flink-connector-elasticsearch-base-1.5.4.jar (Please note that the software have been modified.) License : https://github.com/apache/flink/blob/master/LICENSE +1.3.29 inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/reader/KafkaSourceReader.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSource.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/source/KafkaSourceBuilder.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaDeserializationSchema.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/DynamicKafkaRecordSerializationSchema.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptions.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaConnectorOptionsUtil.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSink.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicSource.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/KafkaDynamicTableFactory.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertSink.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/ReducingUpsertWriter.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/SinkBufferFlushMode.java + inlong-sort/sort-flink/sort-flink-v1.18/sort-connectors/kafka/src/main/java/org/apache/inlong/sort/kafka/table/UpsertKafkaDynamicTableFactory.java +Source : org.apache.flink:flink-connector-kafka:3.2.0 (Please note that the software have been modified.) +License : https://github.com/apache/flink-connector-kafka/blob/main/LICENSE + ======================================================================= Apache InLong Subcomponents: diff --git a/pom.xml b/pom.xml index 43303600331..0562af4dd73 100644 --- a/pom.xml +++ b/pom.xml @@ -190,6 +190,7 @@ 4.0.3 2.9.3 3.0.0 + 5.6.54 1.2.5 6.0.0