Skip to content

Commit

Permalink
[CALCITE-2913] Adapter for Apache Kafka (Mingmin Xu)
Browse files Browse the repository at this point in the history
Expose an Apache Kafka topic as a stream table.
  • Loading branch information
mingmxu committed May 15, 2019
1 parent d0a06f7 commit 12d6d8c
Show file tree
Hide file tree
Showing 17 changed files with 1,046 additions and 1 deletion.
99 changes: 99 additions & 0 deletions kafka/pom.xml
@@ -0,0 +1,99 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
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:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
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.calcite</groupId>
<artifactId>calcite</artifactId>
<version>1.20.0-SNAPSHOT</version>
</parent>

<!-- The basics. -->
<artifactId>calcite-kafka</artifactId>
<packaging>jar</packaging>
<name>calcite kafka</name>
<description>Kafka Adapter. Exposes kafka topic(s) as stream table(s).</description>

<properties>
<top.dir>${project.basedir}/..</top.dir>
<build.timestamp>${maven.build.timestamp}</build.timestamp>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-core</artifactId>
<type>test-jar</type>
<scope>test</scope>
<exclusions>
<exclusion>
<artifactId>commons-compiler</artifactId>
<groupId>org.codehaus.janino</groupId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.calcite</groupId>
<artifactId>calcite-linq4j</artifactId>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
<version>${maven-dependency-plugin.version}</version>
<executions>
<execution>
<id>copy-dependencies</id>
<phase>package</phase>
<goals>
<goal>copy-dependencies</goal>
</goals>
<configuration>
<outputDirectory>${project.build.directory}/dependencies/</outputDirectory>
<overWriteReleases>false</overWriteReleases>
<overWriteSnapshots>false</overWriteSnapshots>
<overWriteIfNewer>true</overWriteIfNewer>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>

</project>
@@ -0,0 +1,91 @@
/*
* 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.calcite.adapter.kafka;

import org.apache.calcite.linq4j.Enumerator;

import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;

import java.time.Duration;
import java.util.LinkedList;
import java.util.concurrent.atomic.AtomicBoolean;

/**
* Enumerator to read data from {@link Consumer},
* and converted into SQL rows with {@link KafkaRowConverter}.
* @param <K>: type for Kafka message key,
* refer to {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG};
* @param <V>: type for Kafka message value,
* refer to {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG};
*/
public class KafkaMessageEnumerator<K, V> implements Enumerator<Object[]> {
final Consumer consumer;
final KafkaRowConverter<K, V> rowConverter;
private final AtomicBoolean cancelFlag;

//runtime
private final LinkedList<ConsumerRecord<K, V>> bufferedRecords = new LinkedList<>();
private ConsumerRecord<K, V> curRecord;

KafkaMessageEnumerator(final Consumer consumer,
final KafkaRowConverter<K, V> rowConverter,
final AtomicBoolean cancelFlag) {
this.consumer = consumer;
this.rowConverter = rowConverter;
this.cancelFlag = cancelFlag;
}

/**
* It returns an Array of Object, with each element represents a field of row.
*/
@Override public Object[] current() {
return rowConverter.toRow(curRecord);
}

@Override public boolean moveNext() {
if (cancelFlag.get()) {
return false;
}

while (bufferedRecords.isEmpty()) {
pullRecords();
}

curRecord = bufferedRecords.removeFirst();
return true;
}

private void pullRecords() {
ConsumerRecords<K, V> records = consumer.poll(Duration.ofMillis(100));
for (ConsumerRecord record : records) {
bufferedRecords.add(record);
}
}

@Override public void reset() {
this.bufferedRecords.clear();
pullRecords();
}

@Override public void close() {
consumer.close();
}
}
// End KafkaMessageEnumerator.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.calcite.adapter.kafka;

import org.apache.calcite.rel.type.RelDataType;

import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;

/**
* Interface to handle formatting between Kafka message and Calcite row.
*
* @param <K>: type for Kafka message key,
* refer to {@link ConsumerConfig#KEY_DESERIALIZER_CLASS_CONFIG};
* @param <V>: type for Kafka message value,
* refer to {@link ConsumerConfig#VALUE_DESERIALIZER_CLASS_CONFIG};
*
*/
public interface KafkaRowConverter<K, V> {

/**
* Generate row type for a given Kafka topic.
*
* @param topicName, Kafka topic name;
* @return row type
*/
RelDataType rowDataType(String topicName);

/**
* Parse and reformat Kafka message from consumer,
* to align with row type defined as {@link #rowDataType(String)}.
* @param message, the raw Kafka message record;
* @return fields in the row
*/
Object[] toRow(ConsumerRecord<K, V> message);
}
// End KafkaRowConverter.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.calcite.adapter.kafka;

import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rel.type.RelDataTypeSystem;
import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
import org.apache.calcite.sql.type.SqlTypeName;

import org.apache.kafka.clients.consumer.ConsumerRecord;

/**
* Default implementation of {@link KafkaRowConverter}, both key and value are byte[].
*/
public class KafkaRowConverterImpl implements KafkaRowConverter<byte[], byte[]> {
/**
* Generate row schema for a given Kafka topic.
*
* @param topicName, Kafka topic name;
* @return row type
*/
@Override public RelDataType rowDataType(final String topicName) {
final RelDataTypeFactory typeFactory =
new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
final RelDataTypeFactory.Builder fieldInfo = typeFactory.builder();
fieldInfo.add("MSG_PARTITION", typeFactory.createSqlType(SqlTypeName.INTEGER)).nullable(false);
fieldInfo.add("MSG_TIMESTAMP", typeFactory.createSqlType(SqlTypeName.BIGINT)).nullable(false);
fieldInfo.add("MSG_OFFSET", typeFactory.createSqlType(SqlTypeName.BIGINT)).nullable(false);
fieldInfo.add("MSG_KEY_BYTES", typeFactory.createSqlType(SqlTypeName.VARBINARY)).nullable(true);
fieldInfo.add("MSG_VALUE_BYTES", typeFactory.createSqlType(SqlTypeName.VARBINARY))
.nullable(false);

return fieldInfo.build();
}

/**
* Parse and reformat Kafka message from consumer, to align with row schema
* defined as {@link #rowDataType(String)}.
* @param message, the raw Kafka message record;
* @return fields in the row
*/
@Override public Object[] toRow(final ConsumerRecord<byte[], byte[]> message) {
Object[] fields = new Object[5];
fields[0] = message.partition();
fields[1] = message.timestamp();
fields[2] = message.offset();
fields[3] = message.key();
fields[4] = message.value();

return fields;
}
}
// End KafkaRowConverterImpl.java

0 comments on commit 12d6d8c

Please sign in to comment.