Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[INLONG-10006][Agent] Add MongoDB data source for Agent #10007

Merged
merged 5 commits into from
Apr 18, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -29,24 +29,8 @@ public class MongoTask {
private String user;
private String password;
private String databaseIncludeList;
private String databaseExcludeList;
private String collectionIncludeList;
private String collectionExcludeList;
private String fieldExcludeList;
private String connectTimeoutInMs;
private String queueSize;
private String cursorMaxAwaitTimeInMs;
private String socketTimeoutInMs;
private String selectionTimeoutInMs;
private String fieldRenames;
private String membersAutoDiscover;
private String connectMaxAttempts;
private String connectBackoffMaxDelayInMs;
private String connectBackoffInitialDelayInMs;
private String initialSyncMaxThreads;
private String sslInvalidHostnameAllowed;
private String sslEnabled;
private String pollIntervalInMs;
private String snapshotMode;
private Snapshot snapshot;
private Capture capture;
private Offset offset;
Expand Down Expand Up @@ -87,25 +71,8 @@ public static class MongoTaskConfig {
private String username;
private String password;

private String databaseIncludeList;
private String databaseExcludeList;
private String collectionIncludeList;
private String collectionExcludeList;
private String fieldExcludeList;
private String connectTimeoutInMs;
private String queueSize;
private String cursorMaxAwaitTimeInMs;
private String socketTimeoutInMs;
private String selectionTimeoutInMs;
private String fieldRenames;
private String membersAutoDiscover;
private String connectMaxAttempts;
private String connectBackoffMaxDelayInMs;
private String connectBackoffInitialDelayInMs;
private String initialSyncMaxThreads;
private String sslInvalidHostnameAllowed;
private String sslEnabled;
private String pollIntervalInMs;
private String database;
private String collection;

private String snapshotMode;
private String captureMode;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ 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_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";
Expand Down Expand Up @@ -283,25 +284,9 @@ private static MongoTask getMongoTask(DataConfig dataConfigs) {
mongoTask.setHosts(config.getHosts());
mongoTask.setUser(config.getUsername());
mongoTask.setPassword(config.getPassword());
mongoTask.setDatabaseIncludeList(config.getDatabaseIncludeList());
mongoTask.setDatabaseExcludeList(config.getDatabaseExcludeList());
mongoTask.setCollectionIncludeList(config.getCollectionIncludeList());
mongoTask.setCollectionExcludeList(config.getCollectionExcludeList());
mongoTask.setFieldExcludeList(config.getFieldExcludeList());
mongoTask.setConnectTimeoutInMs(config.getConnectTimeoutInMs());
mongoTask.setQueueSize(config.getQueueSize());
mongoTask.setCursorMaxAwaitTimeInMs(config.getCursorMaxAwaitTimeInMs());
mongoTask.setSocketTimeoutInMs(config.getSocketTimeoutInMs());
mongoTask.setSelectionTimeoutInMs(config.getSelectionTimeoutInMs());
mongoTask.setFieldRenames(config.getFieldRenames());
mongoTask.setMembersAutoDiscover(config.getMembersAutoDiscover());
mongoTask.setConnectMaxAttempts(config.getConnectMaxAttempts());
mongoTask.setConnectBackoffMaxDelayInMs(config.getConnectBackoffMaxDelayInMs());
mongoTask.setConnectBackoffInitialDelayInMs(config.getConnectBackoffInitialDelayInMs());
mongoTask.setInitialSyncMaxThreads(config.getInitialSyncMaxThreads());
mongoTask.setSslInvalidHostnameAllowed(config.getSslInvalidHostnameAllowed());
mongoTask.setSslEnabled(config.getSslEnabled());
mongoTask.setPollIntervalInMs(config.getPollIntervalInMs());
mongoTask.setDatabaseIncludeList(config.getDatabase());
mongoTask.setCollectionIncludeList(config.getCollection());
mongoTask.setSnapshotMode(config.getSnapshotMode());

MongoTask.Offset offset = new MongoTask.Offset();
offset.setFilename(config.getOffsetFilename());
Expand Down Expand Up @@ -511,6 +496,7 @@ public static TaskProfile convertToTaskProfile(DataConfig dataConfig) {
profileDto.setTask(task);
break;
case MONGODB:
task.setTaskClass(DEFAULT_MONGODB_TASK);
MongoTask mongoTask = getMongoTask(dataConfig);
task.setMongoTask(mongoTask);
task.setSource(MONGO_SOURCE);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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.instance;

import org.apache.inlong.agent.conf.InstanceProfile;
import org.apache.inlong.agent.constant.TaskConstants;

public class MongoDBInstance extends CommonInstance {

@Override
public void setInodeInfo(InstanceProfile profile) {
profile.set(TaskConstants.INODE_INFO, "");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,56 +17,159 @@

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.TaskProfile;
import org.apache.inlong.agent.plugin.Message;
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.file.Reader;
import org.apache.inlong.agent.plugin.sources.file.AbstractSource;
import org.apache.inlong.agent.plugin.sources.reader.MongoDBReader;

import io.debezium.connector.mongodb.MongoDbConnector;
import io.debezium.connector.mongodb.MongoDbConnectorConfig;
import io.debezium.engine.ChangeEvent;
import io.debezium.engine.DebeziumEngine;
import io.debezium.engine.DebeziumEngine.RecordCommitter;
import io.debezium.engine.format.Json;
import io.debezium.engine.spi.OffsetCommitPolicy;
import org.apache.kafka.connect.storage.FileOffsetBackingStore;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.Collections;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;

/**
* MongoDBSource : mongo source, split mongo source job into multi readers
*/
public class MongoDBSource extends AbstractSource {

private static final Logger LOGGER = LoggerFactory.getLogger(MongoDBSource.class);
private static final Integer DEBEZIUM_QUEUE_SIZE = 100;
private ExecutorService executor;
public InstanceProfile profile;
private BlockingQueue<SourceData> debeziumQueue;
private final Properties props = new Properties();
private String database;
private String collection;
private String snapshotMode;

private boolean isRestoreFromDB = false;

public MongoDBSource() {
}

@Override
public void init(InstanceProfile profile) {
try {
LOGGER.info("MongoDBSource init: {}", profile.toJsonStr());
this.profile = profile;
super.init(profile);
debeziumQueue = new LinkedBlockingQueue<>(DEBEZIUM_QUEUE_SIZE);
database = profile.get(TaskConstants.TASK_MONGO_DATABASE_INCLUDE_LIST);
collection = profile.get(TaskConstants.TASK_MONGO_COLLECTION_INCLUDE_LIST);
snapshotMode = profile.get(TaskConstants.TASK_MONGO_SNAPSHOT_MODE, "initial");

props.setProperty("name", "MongoDB-" + instanceId);
props.setProperty("connector.class", MongoDbConnector.class.getName());
props.setProperty("offset.storage", FileOffsetBackingStore.class.getName());
String agentPath = AgentConfiguration.getAgentConf()
.get(AgentConstants.AGENT_HOME, AgentConstants.DEFAULT_AGENT_HOME);
String offsetPath = agentPath + "/" + getThreadName() + "offset.dat";
props.setProperty("offset.storage.file.filename", offsetPath);

props.setProperty(String.valueOf(MongoDbConnectorConfig.LOGICAL_NAME), "agent-mongoDB-" + instanceId);
props.setProperty(String.valueOf(MongoDbConnectorConfig.HOSTS),
profile.get(TaskConstants.TASK_MONGO_HOSTS));
props.setProperty(String.valueOf(MongoDbConnectorConfig.USER), profile.get(TaskConstants.TASK_MONGO_USER));
props.setProperty(String.valueOf(MongoDbConnectorConfig.PASSWORD),
profile.get(TaskConstants.TASK_MONGO_PASSWORD));
props.setProperty(String.valueOf(MongoDbConnectorConfig.DATABASE_INCLUDE_LIST), database);
props.setProperty(String.valueOf(MongoDbConnectorConfig.COLLECTION_INCLUDE_LIST), collection);
props.setProperty(String.valueOf(MongoDbConnectorConfig.SNAPSHOT_MODE), snapshotMode);

executor = Executors.newSingleThreadExecutor();
executor.execute(startDebeziumEngine());

} catch (Exception ex) {
stopRunning();
throw new FileException("error init stream for " + collection, ex);
}
}

private Runnable startDebeziumEngine() {
return () -> {
AgentThreadFactory.nameThread(getThreadName() + "debezium");
try (DebeziumEngine<ChangeEvent<String, String>> debeziumEngine = DebeziumEngine.create(Json.class)
.using(props)
.using(OffsetCommitPolicy.always())
.notifying(this::handleConsumerEvent)
.build()) {

debeziumEngine.run();
} catch (Throwable e) {
LOGGER.error("do run error in mongoDB debezium: ", e);
}
};
}

private void handleConsumerEvent(List<ChangeEvent<String, String>> records,
RecordCommitter<ChangeEvent<String, String>> committer) throws InterruptedException {
boolean offerSuc = false;
for (ChangeEvent<String, String> record : records) {
SourceData sourceData = new SourceData(record.value().getBytes(StandardCharsets.UTF_8), 0L);
while (isRunnable() && !offerSuc) {
offerSuc = debeziumQueue.offer(sourceData, 1, TimeUnit.SECONDS);
}
committer.markProcessed(record);
}
committer.markBatchFinished();
}

@Override
public List<Reader> split(TaskProfile conf) {
MongoDBReader mongoDBReader = new MongoDBReader();
List<Reader> readerList = Collections.singletonList(mongoDBReader);
sourceMetric.sourceSuccessCount.incrementAndGet();
return readerList;
return null;
}

@Override
protected String getThreadName() {
return null;
return "mongo-source-" + taskId + "-" + instanceId;
}

@Override
protected void printCurrentState() {

LOGGER.info("mongo collection is {}", collection);
}

@Override
protected boolean doPrepareToRead() {
return false;
return true;
}

@Override
protected List<SourceData> readFromSource() {
return null;
}

@Override
public Message read() {
return null;
List<SourceData> dataList = new ArrayList<>();
try {
int size = 0;
while (size < BATCH_READ_LINE_TOTAL_LEN) {
SourceData sourceData = debeziumQueue.poll(1, TimeUnit.SECONDS);
if (sourceData != null) {
size += sourceData.getData().length;
dataList.add(sourceData);
} else {
break;
}
}
} catch (InterruptedException e) {
LOGGER.error("poll {} data from debezium queue interrupted.", instanceId);
}
return dataList;
}

@Override
Expand All @@ -76,16 +179,12 @@ protected boolean isRunnable() {

@Override
protected void releaseSource() {

}

@Override
public boolean sourceFinish() {
return false;
LOGGER.info("release mongo source");
executor.shutdownNow();
}

@Override
public boolean sourceExist() {
return false;
return true;
}
}
Loading
Loading