提交 2ecd03ab 编写于 作者: A Ali Ahmed 提交者: Matteo Merli

Reintroduce Aerospike connector (#2524)

上级 669196c2
......@@ -74,15 +74,23 @@
<outputDirectory>connectors</outputDirectory>
<fileMode>644</fileMode>
</file>
<file>
<source>${basedir}/../../pulsar-io/jdbc/target/pulsar-io-jdbc-${project.version}.nar</source>
<outputDirectory>connectors</outputDirectory>
<fileMode>644</fileMode>
</file>
<file>
<source>${basedir}/../../pulsar-io/data-genenator/target/pulsar-io-data-generator-${project.version}.nar</source>
<outputDirectory>connectors</outputDirectory>
<fileMode>644</fileMode>
</file>
<file>
<source>${basedir}/../../pulsar-io/aerospike/target/pulsar-io-aerospike-${project.version}.nar</source>
<outputDirectory>connectors</outputDirectory>
<fileMode>644</fileMode>
</file>
</files>
</assembly>
......@@ -162,6 +162,7 @@ flexible messaging model and an intuitive client API.</description>
<sketches.version>0.8.3</sketches.version>
<hbc-core.version>2.2.0</hbc-core.version>
<cassandra-driver-core.version>3.4.0</cassandra-driver-core.version>
<aerospike-client.version>4.1.11</aerospike-client.version>
<kafka-client.version>0.10.2.1</kafka-client.version>
<rabbitmq-client.version>5.1.1</rabbitmq-client.version>
<aws-sdk.version>1.11.297</aws-sdk.version>
......
<!--
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.2.0-incubating-SNAPSHOT</version>
</parent>
<artifactId>pulsar-io-aerospike</artifactId>
<name>Pulsar IO :: Aerospike</name>
<dependencies>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-io-core</artifactId>
<version>${project.version}</version>
</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.aerospike</groupId>
<artifactId>aerospike-client-bc</artifactId>
<version>${aerospike-client.version}</version>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-maven-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>
/**
* 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.aerospike;
import com.aerospike.client.AerospikeClient;
import com.aerospike.client.AerospikeException;
import com.aerospike.client.Bin;
import com.aerospike.client.Host;
import com.aerospike.client.Key;
import com.aerospike.client.Value;
import com.aerospike.client.async.EventLoop;
import com.aerospike.client.async.EventPolicy;
import com.aerospike.client.async.NioEventLoops;
import com.aerospike.client.listener.WriteListener;
import com.aerospike.client.policy.ClientPolicy;
import com.aerospike.client.policy.WritePolicy;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingDeque;
import org.apache.pulsar.functions.api.Record;
import org.apache.pulsar.io.core.KeyValue;
import org.apache.pulsar.io.core.Sink;
import org.apache.pulsar.io.core.SinkContext;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A Simple abstract class for Aerospike sink
* Users need to implement extractKeyValue function to use this sink
*/
public abstract class AerospikeAbstractSink<K, V> implements Sink<byte[]> {
private static final Logger LOG = LoggerFactory.getLogger(AerospikeAbstractSink.class);
// ----- Runtime fields
private AerospikeSinkConfig aerospikeSinkConfig;
private AerospikeClient client;
private WritePolicy writePolicy;
private BlockingQueue<AWriteListener> queue;
private NioEventLoops eventLoops;
private EventLoop eventLoop;
@Override
public void open(Map<String, Object> config, SinkContext sinkContext) throws Exception {
aerospikeSinkConfig = AerospikeSinkConfig.load(config);
if (aerospikeSinkConfig.getSeedHosts() == null
|| aerospikeSinkConfig.getKeyspace() == null
|| aerospikeSinkConfig.getColumnName() == null) {
throw new IllegalArgumentException("Required property not set.");
}
writePolicy = new WritePolicy();
writePolicy.maxRetries = aerospikeSinkConfig.getRetries();
writePolicy.setTimeout(aerospikeSinkConfig.getTimeoutMs());
createClient();
queue = new LinkedBlockingDeque<>(aerospikeSinkConfig.getMaxConcurrentRequests());
for (int i = 0; i < aerospikeSinkConfig.getMaxConcurrentRequests(); ++i) {
queue.put(new AWriteListener(queue));
}
eventLoops = new NioEventLoops(new EventPolicy(), 1);
eventLoop = eventLoops.next();
}
@Override
public void close() throws Exception {
if (client != null) {
client.close();
}
if (eventLoops != null) {
eventLoops.close();
}
LOG.info("Connection Closed");
}
@Override
public void write(Record<byte[]> record) {
KeyValue<K, V> keyValue = extractKeyValue(record);
Key key = new Key(aerospikeSinkConfig.getKeyspace(), aerospikeSinkConfig.getKeySet(), keyValue.getKey().toString());
Bin bin = new Bin(aerospikeSinkConfig.getColumnName(), Value.getAsBlob(keyValue.getValue()));
AWriteListener listener = null;
try {
listener = queue.take();
} catch (InterruptedException ex) {
record.fail();
return;
}
listener.setContext(record);
client.put(eventLoop, listener, writePolicy, key, bin);
}
private void createClient() {
String[] hosts = aerospikeSinkConfig.getSeedHosts().split(",");
if (hosts.length <= 0) {
throw new RuntimeException("Invalid Seed Hosts");
}
Host[] aeroSpikeHosts = new Host[hosts.length];
for (int i = 0; i < hosts.length; ++i) {
String[] hostPort = hosts[i].split(":");
aeroSpikeHosts[i] = new Host(hostPort[0], Integer.valueOf(hostPort[1]));
}
ClientPolicy policy = new ClientPolicy();
if (aerospikeSinkConfig.getUserName() != null && !aerospikeSinkConfig.getUserName().isEmpty()
&& aerospikeSinkConfig.getPassword() != null && !aerospikeSinkConfig.getPassword().isEmpty()) {
policy.user = aerospikeSinkConfig.getUserName();
policy.password = aerospikeSinkConfig.getPassword();
}
client = new AerospikeClient(policy, aeroSpikeHosts);
}
private class AWriteListener implements WriteListener {
private Record<byte[]> context;
private BlockingQueue<AWriteListener> queue;
public AWriteListener(BlockingQueue<AWriteListener> queue) {
this.queue = queue;
}
public void setContext(Record<byte[]> record) {
this.context = record;
}
@Override
public void onSuccess(Key key) {
if (context != null) {
context.ack();
}
try {
queue.put(this);
} catch (InterruptedException ex) {
throw new RuntimeException("Interrupted while being added to the queue" ,ex);
}
}
@Override
public void onFailure(AerospikeException e) {
if (context != null) {
context.fail();
}
try {
queue.put(this);
} catch (InterruptedException ex) {
throw new RuntimeException("Interrupted while being added to the queue", ex);
}
}
}
public abstract KeyValue<K, V> extractKeyValue(Record<byte[]> message);
}
\ No newline at end of file
/**
* 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.aerospike;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
import lombok.*;
import lombok.experimental.Accessors;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.util.Map;
@Data
@Setter
@Getter
@EqualsAndHashCode
@ToString
@Accessors(chain = true)
public class AerospikeSinkConfig implements Serializable {
private static final long serialVersionUID = 1L;
private String seedHosts;
private String keyspace;
private String columnName;
// Optional
private String userName;
private String password;
private String keySet;
private int maxConcurrentRequests = 100;
private int timeoutMs = 100;
private int retries = 1;
public static AerospikeSinkConfig load(String yamlFile) throws IOException {
ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
return mapper.readValue(new File(yamlFile), AerospikeSinkConfig.class);
}
public static AerospikeSinkConfig load(Map<String, Object> map) throws IOException {
ObjectMapper mapper = new ObjectMapper();
return mapper.readValue(new ObjectMapper().writeValueAsString(map), AerospikeSinkConfig.class);
}
}
\ No newline at end of file
/**
* 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.aerospike;
import org.apache.pulsar.functions.api.Record;
import org.apache.pulsar.io.core.KeyValue;
/**
* Aerospike sink that treats incoming messages on the input topic as Strings
* and write identical key/value pairs.
*/
public class AerospikeStringSink extends AerospikeAbstractSink<String, String> {
@Override
public KeyValue<String, String> extractKeyValue(Record<byte[]> record) {
String key = record.getKey().orElseGet(() -> new String(record.getValue()));
return new KeyValue<>(key, new String(record.getValue()));
}
}
\ No newline at end of file
#
# 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.
#
name: aerospike
description: Aerospike database sink
sinkClass: org.apache.pulsar.io.aerospike.AerospikeStringSink
......@@ -35,6 +35,7 @@
<module>core</module>
<module>twitter</module>
<module>cassandra</module>
<module>aerospike</module>
<module>kafka</module>
<module>rabbitmq</module>
<module>kinesis</module>
......
......@@ -125,6 +125,7 @@ $ tar xvfz apache-pulsar-io-connectors-{{pulsar:version}}-bin.tar.gz
$ mv apache-pulsar-io-connectors-{{pulsar:version}}/connectors connectors
$ ls connectors
pulsar-io-aerospike-{{pulsar:version}}.nar
pulsar-io-cassandra-{{pulsar:version}}.nar
pulsar-io-kafka-{{pulsar:version}}.nar
pulsar-io-kinesis-{{pulsar:version}}.nar
......
......@@ -87,6 +87,7 @@ $ tar xvfz /path/to/apache-pulsar-io-connectors-{{pulsar:version}}-bin.tar.gz
$ cd apache-pulsar-io-connectors-{{pulsar:version}}/connectors connectors
$ ls connectors
pulsar-io-aerospike-{{pulsar:version}}.nar
pulsar-io-cassandra-{{pulsar:version}}.nar
pulsar-io-kafka-{{pulsar:version}}.nar
pulsar-io-kinesis-{{pulsar:version}}.nar
......
---
id: io-aerospike
title: Aerospike Sink Connector
sidebar_label: Aerospike Sink Connector
---
The Aerospike Sink connector is used to write messages to an Aerospike Cluster.
## Sink Configuration Options
The following configuration options are specific to the Aerospike Connector:
| Name | Required | Default | Description |
|------|----------|---------|-------------|
| `seedHosts` | `true` | `null` | Comma seperated list of one or more Aerospike cluster hosts; each host can be specified as a valid IP address or hostname followed by an optional port number (default is 3000). |
| `keyspace` | `true` | `null` | Aerospike namespace to use. |
| `keySet` | `false` | `null` | Aerospike set name to use. |
| `columnName` | `true` | `null` | Aerospike bin name to use. |
| `maxConcurrentRequests` | `false` | `100` | Maximum number of concurrent Aerospike transactions that a Sink can open. |
| `timeoutMs` | `false` | `100` | A single timeout value controls `socketTimeout` and `totalTimeout` for Aerospike transactions. |
| `retries` | `false` | `1` | Maximum number of retries before aborting a write transaction to Aerospike. |
......@@ -9,6 +9,7 @@ These connectors import and export data from some of the most commonly used data
as easy as writing a simple connector configuration and running the connector locally or submitting the connector to a
Pulsar Functions cluster.
- [Aerospike Sink Connector](io-aerospike.md)
- [Cassandra Sink Connector](io-cassandra.md)
- [Kafka Sink Connector](io-kafka.md#sink)
- [Kafka Source Connector](io-kafka.md#source)
......
......@@ -4,7 +4,7 @@ title: Pulsar IO Overview
sidebar_label: Overview
---
Messaging systems are most powerful when you can easily use them in conjunction with external systems like databases and other messaging systems. **Pulsar IO** is a feature of Pulsar that enables you to easily create, deploy, and manage Pulsar **connectors** that interact with external systems, such as [Apache Cassandra](https://cassandra.apache.org), and many others.
Messaging systems are most powerful when you can easily use them in conjunction with external systems like databases and other messaging systems. **Pulsar IO** is a feature of Pulsar that enables you to easily create, deploy, and manage Pulsar **connectors** that interact with external systems, such as [Apache Cassandra](https://cassandra.apache.org), [Aerospike](https://www.aerospike.com), and many others.
> #### Pulsar IO and Pulsar Functions
> Under the hood, Pulsar IO connectors are specialized [Pulsar Functions](functions-overview.md) purpose-built to interface with external systems. The [administrative interface](io-quickstart.md) for Pulsar IO is, in fact, quite similar to that of Pulsar Functions.
......@@ -30,6 +30,7 @@ The following connectors are currently available for Pulsar:
|Name|Java Class|Documentation|
|---|---|---|
|[Aerospike sink](https://www.aerospike.com/)|[`org.apache.pulsar.io.aerospike.AerospikeSink`](https://github.com/apache/incubator-pulsar/blob/master/pulsar-io/aerospike/src/main/java/org/apache/pulsar/io/aerospike/AerospikeStringSink.java)|[Documentation](io-aerospike.md)|
|[Cassandra sink](https://cassandra.apache.org)|[`org.apache.pulsar.io.cassandra.CassandraSink`](https://github.com/apache/incubator-pulsar/blob/master/pulsar-io/cassandra/src/main/java/org/apache/pulsar/io/cassandra/CassandraStringSink.java)|[Documentation](io-cassandra.md)|
|[Kafka source](https://kafka.apache.org)|[`org.apache.pulsar.io.kafka.KafkaSource`](https://github.com/apache/incubator-pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaStringSource.java)|[Documentation](io-kafka.md#source)|
|[Kafka sink](https://kafka.apache.org)|[`org.apache.pulsar.io.kafka.KafkaSink`](https://github.com/apache/incubator-pulsar/blob/master/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaStringSink.java)|[Documentation](io-kafka.md#sink)|
......
......@@ -69,6 +69,7 @@ $ tar xvfz /path/to/apache-pulsar-io-connectors-{{pulsar:version}}-bin.tar.gz
$ cp -r apache-pulsar-io-connectors-{{pulsar:version}}/connectors connectors
$ ls connectors
pulsar-io-aerospike-{{pulsar:version}}.nar
pulsar-io-cassandra-{{pulsar:version}}.nar
pulsar-io-kafka-{{pulsar:version}}.nar
pulsar-io-kinesis-{{pulsar:version}}.nar
......@@ -122,7 +123,7 @@ curl -s http://localhost:8080/admin/v2/functions/connectors
Example output:
```json
[{"name":"cassandra","description":"Writes data into Cassandra","sinkClass":"org.apache.pulsar.io.cassandra.CassandraStringSink"},{"name":"kafka","description":"Kafka source and sink connector","sourceClass":"org.apache.pulsar.io.kafka.KafkaStringSource","sinkClass":"org.apache.pulsar.io.kafka.KafkaStringSink"},{"name":"kinesis","description":"Kinesis sink connector","sinkClass":"org.apache.pulsar.io.kinesis.KinesisSink"},{"name":"rabbitmq","description":"RabbitMQ source connector","sourceClass":"org.apache.pulsar.io.rabbitmq.RabbitMQSource"},{"name":"twitter","description":"Ingest data from Twitter firehose","sourceClass":"org.apache.pulsar.io.twitter.TwitterFireHose"}]
[{"name":"aerospike","description":"Aerospike database sink","sinkClass":"org.apache.pulsar.io.aerospike.AerospikeStringSink"},{"name":"cassandra","description":"Writes data into Cassandra","sinkClass":"org.apache.pulsar.io.cassandra.CassandraStringSink"},{"name":"kafka","description":"Kafka source and sink connector","sourceClass":"org.apache.pulsar.io.kafka.KafkaStringSource","sinkClass":"org.apache.pulsar.io.kafka.KafkaStringSink"},{"name":"kinesis","description":"Kinesis sink connector","sinkClass":"org.apache.pulsar.io.kinesis.KinesisSink"},{"name":"rabbitmq","description":"RabbitMQ source connector","sourceClass":"org.apache.pulsar.io.rabbitmq.RabbitMQSource"},{"name":"twitter","description":"Ingest data from Twitter firehose","sourceClass":"org.apache.pulsar.io.twitter.TwitterFireHose"}]
```
If an error occurred while starting Pulsar service, you may be able to seen exception at the terminal you are running `pulsar/standalone`,
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册