Skip to content

Commit

Permalink
Add a Pulsar IO MongoDB (apache#3561)
Browse files Browse the repository at this point in the history
### Motivation

Provides a builtin MongoDB Connector, in order to ease the storage of JSON formated message in MongoDB. It's a sink connector.


### Modifications

Add a new sub-module in the `pulsar-io`module.

### Verifying this change

This change added tests and can be verified as follows:
* deploy the connector with configuration file containing the following fields:
```
configs:
  mongoUri: mongodb://hostname:port
  database: pulsar
  collection: messages
```
* start a mongodb instance
* send messages in the topic declared when deploying the connector
* check in MongoDB if the messages have been stored in the collection `messages`
  • Loading branch information
bbonnin authored and sijie committed Feb 13, 2019
1 parent 3aeafc1 commit 06e911d
Show file tree
Hide file tree
Showing 10 changed files with 762 additions and 0 deletions.
82 changes: 82 additions & 0 deletions pulsar-io/mongo/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
<!--
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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>

<parent>
<groupId>org.apache.pulsar</groupId>
<artifactId>pulsar-io</artifactId>
<version>2.3.0-SNAPSHOT</version>
</parent>

<artifactId>pulsar-io-mongo</artifactId>
<name>Pulsar IO :: MongoDB</name>

<properties>
<mongo-driver.version>3.8.2</mongo-driver.version>
</properties>

<dependencies>
<dependency>
<groupId>${project.parent.groupId}</groupId>
<artifactId>pulsar-io-core</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>org.mongodb</groupId>
<artifactId>mongodb-driver-async</artifactId>
<version>${mongo-driver.version}</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-yaml</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>${project.parent.groupId}</groupId>
<artifactId>buildtools</artifactId>
<version>${project.parent.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-maven-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/**
* 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.pulsar.io.mongodb;

import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
import com.google.common.base.Preconditions;
import lombok.*;
import lombok.experimental.Accessors;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.io.core.annotations.FieldDoc;

import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.util.Map;

/**
* Configuration class for the MongoDB Sink Connector.
*/
@Data
@Setter
@Getter
@EqualsAndHashCode
@ToString
@Accessors(chain = true)
public class MongoConfig implements Serializable {

private static final long serialVersionUID = 1L;

public static final int DEFAULT_BATCH_SIZE = 100;

public static final long DEFAULT_BATCH_TIME_MS = 1000;

@FieldDoc(
required = true,
defaultValue = "",
help = "The uri of mongodb that the connector connects to" +
" (see: https://docs.mongodb.com/manual/reference/connection-string/)"
)
private String mongoUri;

@FieldDoc(
required = true,
defaultValue = "",
help = "The name of the database to which the collection belongs to"
)
private String database;

@FieldDoc(
required = true,
defaultValue = "",
help = "The collection name that the connector writes messages to"
)
private String collection;

@FieldDoc(
required = false,
defaultValue = "" + DEFAULT_BATCH_SIZE,
help = "The batch size of write to the collection"
)
private int batchSize = DEFAULT_BATCH_SIZE;

@FieldDoc(
required = false,
defaultValue = "" + DEFAULT_BATCH_TIME_MS,
help = "The batch operation interval in milliseconds")
private long batchTimeMs = DEFAULT_BATCH_TIME_MS;


public static MongoConfig load(String yamlFile) throws IOException {
final ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
final MongoConfig cfg = mapper.readValue(new File(yamlFile), MongoConfig.class);

return cfg;
}

public static MongoConfig load(Map<String, Object> map) throws IOException {
final ObjectMapper mapper = new ObjectMapper();
final MongoConfig cfg = mapper.readValue(new ObjectMapper().writeValueAsString(map), MongoConfig.class);

return cfg;
}

public void validate() {
if (StringUtils.isEmpty(mongoUri) || StringUtils.isEmpty(database) || StringUtils.isEmpty(collection)) {
throw new IllegalArgumentException("Required property not set.");
}

Preconditions.checkArgument(batchSize > 0, "batchSize must be a positive integer.");
Preconditions.checkArgument(batchTimeMs > 0, "batchTimeMs must be a positive long.");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,182 @@
/**
* 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.pulsar.io.mongodb;

import com.google.common.collect.Lists;
import com.mongodb.MongoBulkWriteException;
import com.mongodb.async.client.MongoClient;
import com.mongodb.async.client.MongoClients;
import com.mongodb.async.client.MongoCollection;
import com.mongodb.async.client.MongoDatabase;
import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.functions.api.Record;
import org.apache.pulsar.io.core.Sink;
import org.apache.pulsar.io.core.SinkContext;
import org.apache.pulsar.io.core.annotations.Connector;
import org.apache.pulsar.io.core.annotations.IOType;
import org.bson.BSONException;
import org.bson.Document;
import org.bson.json.JsonParseException;

import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.stream.IntStream;

import static java.util.stream.Collectors.toList;

/**
* The base class for MongoDB sinks.
* Users need to implement extractKeyValue function to use this sink.
* This class assumes that the input will be JSON documents.
*/
@Connector(
name = "mongo",
type = IOType.SINK,
help = "A sink connector that sends pulsar messages to mongodb",
configClass = MongoConfig.class
)
@Slf4j
public class MongoSink implements Sink<byte[]> {

private MongoConfig mongoConfig;

private MongoClient mongoClient;

private MongoCollection<Document> collection;

private List<Record<byte[]>> incomingList;

private ScheduledExecutorService flushExecutor;


@Override
public void open(Map<String, Object> config, SinkContext sinkContext) throws Exception {
log.info("Open MongoDB Sink");

mongoConfig = MongoConfig.load(config);
mongoConfig.validate();

mongoClient = MongoClients.create(mongoConfig.getMongoUri());
final MongoDatabase db = mongoClient.getDatabase(mongoConfig.getDatabase());
collection = db.getCollection(mongoConfig.getCollection());

incomingList = Lists.newArrayList();
flushExecutor = Executors.newScheduledThreadPool(1);
flushExecutor.scheduleAtFixedRate(() -> flush(),
mongoConfig.getBatchTimeMs(), mongoConfig.getBatchTimeMs(), TimeUnit.MILLISECONDS);
}

@Override
public void write(Record<byte[]> record) {
final String recordValue = new String(record.getValue(), Charset.forName("UTF-8"));

if (log.isDebugEnabled()) {
log.debug("Received record: " + recordValue);
}

int currentSize;

synchronized (this) {
incomingList.add(record);
currentSize = incomingList.size();
}

if (currentSize == mongoConfig.getBatchSize()) {
flushExecutor.submit(() -> flush());
}
}

private void flush() {
final List<Document> docsToInsert = new ArrayList<>();
final List<Record<byte[]>> recordsToInsert;

synchronized (this) {
if (incomingList.isEmpty()) {
return;
}

recordsToInsert = incomingList;
incomingList = Lists.newArrayList();
}

final Iterator<Record<byte[]>> iter = recordsToInsert.iterator();

while (iter.hasNext()) {
final Record<byte[]> record = iter.next();

try {
final byte[] docAsBytes = record.getValue();
final Document doc = Document.parse(new String(docAsBytes, Charset.forName("UTF-8")));
docsToInsert.add(doc);
}
catch (JsonParseException | BSONException e) {
log.error("Bad message", e);
record.fail();
iter.remove();
}
}

if (docsToInsert.size() > 0) {

collection.insertMany(docsToInsert, (result, t) -> {
final List<Integer> idxToAck = IntStream.range(0, docsToInsert.size()).boxed().collect(toList());
final List<Integer> idxToFail = Lists.newArrayList();

if (t != null) {
log.error("MongoDB insertion error", t);

if (t instanceof MongoBulkWriteException) {
// With this exception, we are aware of the items that have not been inserted.
((MongoBulkWriteException) t).getWriteErrors().forEach(err -> {
idxToFail.add(err.getIndex());
});
idxToAck.removeAll(idxToFail);
} else {
idxToFail.addAll(idxToAck);
idxToAck.clear();
}
}

if (log.isDebugEnabled()) {
log.debug("Nb ack={}, nb fail={}", idxToAck.size(), idxToFail.size());
}

idxToAck.forEach(idx -> recordsToInsert.get(idx).ack());
idxToFail.forEach(idx -> recordsToInsert.get(idx).fail());
});
}
}

@Override
public void close() throws Exception {
if (flushExecutor != null) {
flushExecutor.shutdown();
}

if (mongoClient != null) {
mongoClient.close();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,19 @@
/**
* 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.pulsar.io.mongodb;
Loading

0 comments on commit 06e911d

Please sign in to comment.