XuMingmin commented on a change in pull request #1127: [CALCITE-2913] add a KafkaAdapter for Stream URL: https://github.com/apache/calcite/pull/1127#discussion_r283086901
########## File path: kafka/src/main/java/org/apache/calcite/adapter/kafka/KafkaBaseTable.java ########## @@ -0,0 +1,135 @@ +/* + * 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.DataContext; +import org.apache.calcite.config.CalciteConnectionConfig; +import org.apache.calcite.linq4j.AbstractEnumerable; +import org.apache.calcite.linq4j.Enumerable; +import org.apache.calcite.linq4j.Enumerator; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.schema.ScannableTable; +import org.apache.calcite.schema.Statistic; +import org.apache.calcite.schema.Statistics; +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlNode; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; + +import com.google.common.collect.ImmutableList; + +import java.util.Collections; +import java.util.Properties; + +/** + * A table which maps to a Kafka topic, currently only {@link KafkaStreamTable} is implemented as + * a STREAM table. + */ +public abstract class KafkaBaseTable implements ScannableTable { + final KafkaTableOptions tableOptions; + + public KafkaBaseTable(final KafkaTableOptions tableOptions) { + this.tableOptions = tableOptions; + } + + /** Returns an enumerator over the rows in this Table. Each row is represented + * as an array of its column values. + * @param root, refer to {@link DataContext}. + */ + @Override public Enumerable<Object[]> scan(final DataContext root) { + return new AbstractEnumerable<Object[]>() { + public Enumerator<Object[]> enumerator() { + if (tableOptions.consumer() != null) { + return new KafkaMessageEnumerator(tableOptions.consumer(), tableOptions.rowConverter()); + } + + try { + Properties consumerConfig = new Properties(); + consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, + tableOptions.bootstrapServers()); + //by default it's <byte[], byte[]> + consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + + if (tableOptions.consumerParas() != null) { + consumerConfig.putAll(tableOptions.consumerParas()); + } + Consumer consumer = new KafkaConsumer<>(consumerConfig); + consumer.subscribe(Collections.singletonList(tableOptions.topicName())); + + return new KafkaMessageEnumerator(consumer, tableOptions.rowConverter()); + } catch (Exception e) { Review comment: No exception(besides RuntimeException) here actually, removed. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services
