consumerParams) {
+ this.consumerParams = consumerParams;
+ return this;
+ }
+
+ public Consumer getConsumer() {
+ return consumer;
+ }
+
+ public KafkaTableOptions setConsumer(final Consumer consumer) {
+ this.consumer = consumer;
+ return this;
+ }
+}
+// End KafkaTableOptions.java
diff --git a/kafka/src/main/java/org/apache/calcite/adapter/kafka/package-info.java b/kafka/src/main/java/org/apache/calcite/adapter/kafka/package-info.java
new file mode 100644
index 000000000000..a303c7062d45
--- /dev/null
+++ b/kafka/src/main/java/org/apache/calcite/adapter/kafka/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+/**
+ * Kafka query provider.
+ *
+ * One Kafka topic is mapping to one STREAM table.
+ */
+@PackageMarker
+package org.apache.calcite.adapter.kafka;
+
+import org.apache.calcite.avatica.util.PackageMarker;
+
+// End package-info.java
diff --git a/kafka/src/test/java/org/apache/calcite/adapter/kafka/KafkaAdapterTest.java b/kafka/src/test/java/org/apache/calcite/adapter/kafka/KafkaAdapterTest.java
new file mode 100644
index 000000000000..79709cc08ea2
--- /dev/null
+++ b/kafka/src/test/java/org/apache/calcite/adapter/kafka/KafkaAdapterTest.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.calcite.adapter.kafka;
+
+import org.apache.calcite.test.CalciteAssert;
+
+import com.google.common.io.Resources;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+
+/**
+ * Unit test cases for Kafka adapter.
+ */
+public class KafkaAdapterTest {
+ protected static final URL MODEL = KafkaAdapterTest.class.getResource("/kafka.model.json");
+
+ private CalciteAssert.AssertThat assertModel(String model) {
+ // ensure that Schema from this instance is being used
+ model = model.replace(KafkaAdapterTest.class.getName(), KafkaAdapterTest.class.getName());
+
+ return CalciteAssert.that()
+ .withModel(model);
+ }
+
+ private CalciteAssert.AssertThat assertModel(URL url) {
+ Objects.requireNonNull(url, "url");
+ try {
+ return assertModel(Resources.toString(url, StandardCharsets.UTF_8));
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
+ @Test public void testSelect() {
+ assertModel(MODEL)
+ .query("SELECT STREAM * FROM KAFKA.MOCKTABLE")
+ .limit(2)
+
+ .typeIs("[MSG_PARTITION INTEGER NOT NULL"
+ + ", MSG_TIMESTAMP BIGINT NOT NULL"
+ + ", MSG_OFFSET BIGINT NOT NULL"
+ + ", MSG_KEY_BYTES VARBINARY"
+ + ", MSG_VALUE_BYTES VARBINARY NOT NULL]")
+
+ .returnsUnordered(
+ "MSG_PARTITION=0; MSG_TIMESTAMP=-1; MSG_OFFSET=0; MSG_KEY_BYTES=mykey0; MSG_VALUE_BYTES=myvalue0",
+ "MSG_PARTITION=0; MSG_TIMESTAMP=-1; MSG_OFFSET=1"
+ + "; MSG_KEY_BYTES=mykey1; MSG_VALUE_BYTES=myvalue1")
+
+ .explainContains("PLAN=EnumerableInterpreter\n"
+ + " BindableTableScan(table=[[KAFKA, MOCKTABLE, (STREAM)]])\n");
+ }
+
+ @Test public void testFilterWithProject() {
+ assertModel(MODEL)
+ .query("SELECT STREAM MSG_PARTITION,MSG_OFFSET,MSG_VALUE_BYTES FROM KAFKA.MOCKTABLE"
+ + " WHERE MSG_OFFSET>0")
+ .limit(1)
+
+ .returnsUnordered(
+ "MSG_PARTITION=0; MSG_OFFSET=1; MSG_VALUE_BYTES=myvalue1")
+ .explainContains(
+ "PLAN=EnumerableCalc(expr#0..4=[{inputs}], expr#5=[0], expr#6=[>($t2, $t5)], MSG_PARTITION=[$t0], MSG_OFFSET=[$t2], MSG_VALUE_BYTES=[$t4], $condition=[$t6])\n"
+ + " EnumerableInterpreter\n"
+ + " BindableTableScan(table=[[KAFKA, MOCKTABLE, (STREAM)]])");
+ }
+
+ @Test public void testCustRowConverter() {
+ assertModel(MODEL)
+ .query("SELECT STREAM * FROM KAFKA.MOCKTABLE_CUST_ROW_CONVERTER")
+ .limit(2)
+
+ .typeIs("[TOPIC_NAME VARCHAR NOT NULL"
+ + ", PARTITION_ID INTEGER NOT NULL"
+ + ", TIMESTAMP_TYPE VARCHAR]")
+
+ .returnsUnordered(
+ "TOPIC_NAME=testtopic; PARTITION_ID=0; TIMESTAMP_TYPE=NoTimestampType",
+ "TOPIC_NAME=testtopic; PARTITION_ID=0; TIMESTAMP_TYPE=NoTimestampType")
+
+ .explainContains("PLAN=EnumerableInterpreter\n"
+ + " BindableTableScan(table=[[KAFKA, MOCKTABLE_CUST_ROW_CONVERTER, (STREAM)]])\n");
+ }
+
+
+ @Test public void testAsBatch() {
+ assertModel(MODEL)
+ .query("SELECT * FROM KAFKA.MOCKTABLE")
+ .failsAtValidation("Cannot convert stream 'MOCKTABLE' to relation");
+ }
+}
+// End KafkaAdapterTest.java
diff --git a/kafka/src/test/java/org/apache/calcite/adapter/kafka/KafkaMockConsumer.java b/kafka/src/test/java/org/apache/calcite/adapter/kafka/KafkaMockConsumer.java
new file mode 100644
index 000000000000..cccbcfd0dcd9
--- /dev/null
+++ b/kafka/src/test/java/org/apache/calcite/adapter/kafka/KafkaMockConsumer.java
@@ -0,0 +1,48 @@
+/*
+ * 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.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.MockConsumer;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
+import org.apache.kafka.common.TopicPartition;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.HashMap;
+
+/**
+ * A mock consumer to test Kafka adapter.
+ */
+public class KafkaMockConsumer extends MockConsumer {
+ public KafkaMockConsumer(final OffsetResetStrategy offsetResetStrategy) {
+ super(OffsetResetStrategy.EARLIEST);
+
+ assign(Arrays.asList(new TopicPartition("testtopic", 0)));
+
+ HashMap beginningOffsets = new HashMap<>();
+ beginningOffsets.put(new TopicPartition("testtopic", 0), 0L);
+ updateBeginningOffsets(beginningOffsets);
+
+ for (int idx = 0; idx < 10; ++idx) {
+ addRecord(new ConsumerRecord("testtopic",
+ 0, idx, ("mykey" + idx).getBytes(StandardCharsets.UTF_8),
+ ("myvalue" + idx).getBytes(StandardCharsets.UTF_8)));
+ }
+ }
+}
+// End KafkaMockConsumer.java
diff --git a/kafka/src/test/java/org/apache/calcite/adapter/kafka/KafkaRowConverterTest.java b/kafka/src/test/java/org/apache/calcite/adapter/kafka/KafkaRowConverterTest.java
new file mode 100644
index 000000000000..8bbebb487bd5
--- /dev/null
+++ b/kafka/src/test/java/org/apache/calcite/adapter/kafka/KafkaRowConverterTest.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.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;
+
+/**
+ * Implementation of {@link KafkaRowConverter} for test, both key and value are saved as byte[].
+ */
+public class KafkaRowConverterTest implements KafkaRowConverter {
+ /**
+ * 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("TOPIC_NAME", typeFactory.createSqlType(SqlTypeName.VARCHAR)).nullable(false);
+ fieldInfo.add("PARTITION_ID", typeFactory.createSqlType(SqlTypeName.INTEGER)).nullable(false);
+ fieldInfo.add("TIMESTAMP_TYPE", typeFactory.createSqlType(SqlTypeName.VARCHAR)).nullable(true);
+
+ return fieldInfo.build();
+ }
+
+ /**
+ * Parse and reformat Kafka message from consumer, to fit 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 message) {
+ Object[] fields = new Object[4];
+ fields[0] = message.topic();
+ fields[1] = message.partition();
+ fields[2] = message.timestampType().name;
+
+ return fields;
+ }
+}
+// End KafkaRowConverterTest.java
diff --git a/kafka/src/test/resources/kafka.model.json b/kafka/src/test/resources/kafka.model.json
new file mode 100644
index 000000000000..a37d499bcc1b
--- /dev/null
+++ b/kafka/src/test/resources/kafka.model.json
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+{
+ "version": "1.0",
+ "defaultSchema": "KAFKA",
+ "schemas": [
+ {
+ "name": "KAFKA",
+ "tables": [
+ {
+ "name": "MOCKTABLE",
+ "type": "custom",
+ "factory": "org.apache.calcite.adapter.kafka.KafkaTableFactory",
+ "operand": {
+ "consumer.cust": "org.apache.calcite.adapter.kafka.KafkaMockConsumer"
+ }
+ }
+ ,{
+ "name": "MOCKTABLE_CUST_ROW_CONVERTER",
+ "type": "custom",
+ "factory": "org.apache.calcite.adapter.kafka.KafkaTableFactory",
+ "operand": {
+ "consumer.cust": "org.apache.calcite.adapter.kafka.KafkaMockConsumer",
+ "row.converter": "org.apache.calcite.adapter.kafka.KafkaRowConverterTest",
+ "consumer.params": {
+ "key.deserializer": "org.apache.kafka.common.serialization.StringDeserializer",
+ "value.deserializer": "org.apache.kafka.common.serialization.StringDeserializer"
+ }
+ }
+ }
+ ]
+ }
+ ]
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index b5f6161cd650..5e9890099996 100644
--- a/pom.xml
+++ b/pom.xml
@@ -143,6 +143,8 @@ limitations under the License.
0.9.16
2.4.0
+ 2.0.0
+
-html5
@@ -188,6 +190,7 @@ limitations under the License.
spark
splunk
ubenchmark
+ kafka
+
+**Note**:
+
+KafkaAdapter is an experimental feature, changes in public API and usage are expected.
+
+For instructions on downloading and building Calcite, start with the[tutorial]({{ site.baseurl }}/docs/tutorial.html).
+
+The Kafka adapter exposes an Apache Kafka topic as a STREAM table, so it can be queried using
+[Calcite Stream SQL]({{ site.baseurl }}/docs/stream.html). Note that the adapter will not attempt to scan all topics,
+instead users need to configure tables manually, one Kafka stream table is mapping to one Kafka topic.
+
+A basic example of a model file is given below:
+
+{% highlight json %}
+{
+ "version": "1.0",
+ "defaultSchema": "KAFKA",
+ "schemas": [
+ {
+ "name": "KAFKA",
+ "tables": [
+ {
+ "name": "TABLE_NAME",
+ "type": "custom",
+ "factory": "org.apache.calcite.adapter.kafka.KafkaTableFactory",
+ "row.converter": "com.example.CustKafkaRowConverter",
+ "operand": {
+ "bootstrap.servers": "host1:port,host2:port",
+ "topic.name": "kafka.topic.name",
+ "consumer.params": {
+ "key.deserializer": "org.apache.kafka.common.serialization.ByteArrayDeserializer",
+ "value.deserializer": "org.apache.kafka.common.serialization.ByteArrayDeserializer"
+ }
+ }
+ }
+ ]
+ }
+ ]
+}
+{% endhighlight %}
+
+Note that:
+
+1. As Kafka message is schemaless, a [KafkaRowConverter]({{ site.apiRoot }}/org/apache/calcite/adapter/kafka/KafkaRowConverter.html)
+ is required to specify row schema explicitly(with parameter `row.converter`), and
+ how to decode Kafka message to Calcite row. [KafkaRowConverterImpl]({{ site.apiRoot }}/org/apache/calcite/adapter/kafka/KafkaRowConverterImpl.html)
+ is used if not provided;
+
+2. More consumer settings can be added in parameter `consumer.params`;
+
+Assuming this file is stored as `kafka.model.json`, you can connect to Kafka via
+[`sqlline`](https://github.com/julianhyde/sqlline) as follows:
+
+{% highlight bash %}
+$ ./sqlline
+sqlline> !connect jdbc:calcite:model=kafka.model.json admin admin
+{% endhighlight %}
+
+`sqlline` will now accept SQL queries which access your Kafka topics.
+
+With the Kafka table configured in above model. We can run a simple query to fetch messages:
+
+{% highlight sql %}
+sqlline> SELECT STREAM *
+ FROM KAFKA.TABLE_NAME;
++---------------+---------------------+---------------------+---------------+-----------------+
+| MSG_PARTITION | MSG_TIMESTAMP | MSG_OFFSET | MSG_KEY_BYTES | MSG_VALUE_BYTES |
++---------------+---------------------+---------------------+---------------+-----------------+
+| 0 | -1 | 0 | mykey0 | myvalue0 |
+| 0 | -1 | 1 | mykey1 | myvalue1 |
++---------------+---------------------+---------------------+---------------+-----------------+
+{% endhighlight %}
+
+Kafka table is a streaming table, which runs continuously, `LIMIT` can be added to return fast as below:
+
+{% highlight sql %}
+sqlline> SELECT STREAM *
+ FROM KAFKA.TABLE_NAME
+ LIMIT 5;
+{% endhighlight %}
diff --git a/sqlline b/sqlline
index c26f001ba77e..dfabbd03f3b4 100755
--- a/sqlline
+++ b/sqlline
@@ -37,7 +37,7 @@ if [ ! -f target/fullclasspath.txt ]; then
fi
CP=
-for module in core cassandra druid elasticsearch2 elasticsearch5 file mongodb server spark splunk geode example/csv example/function; do
+for module in core cassandra druid elasticsearch2 elasticsearch5 file kafka mongodb server spark splunk geode example/csv example/function; do
CP=${CP}${module}/target/classes:
CP=${CP}${module}/target/test-classes:
done