Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
26 changes: 26 additions & 0 deletions backends-clickhouse/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,32 @@
</dependency>
</dependencies>
</profile>
<profile>
<id>kafka</id>
<activation>
<activeByDefault>false</activeByDefault>
</activation>
<dependencies>
<dependency>
<groupId>org.apache.gluten</groupId>
<artifactId>gluten-kafka</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.gluten</groupId>
<artifactId>gluten-kafka</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql-kafka-0-10_${scala.binary.version}</artifactId>
<version>${spark.version}</version>
<scope>provided</scope>
</dependency>
</dependencies>
</profile>
</profiles>

<dependencies>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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.gluten.component

import org.apache.gluten.backendsapi.clickhouse.CHBackend
import org.apache.gluten.execution.OffloadKafkaScan
import org.apache.gluten.extension.injector.Injector

class CHKafkaComponent extends Component {
override def name(): String = "clickhouse-kafka"
override def buildInfo(): Component.BuildInfo =
Component.BuildInfo("ClickHouseKafka", "N/A", "N/A", "N/A")
override def dependencies(): Seq[Class[_ <: Component]] = classOf[CHBackend] :: Nil
override def injectRules(injector: Injector): Unit = {
OffloadKafkaScan.inject(injector)
}
}
Original file line number Diff line number Diff line change
@@ -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.
*/
package org.apache.gluten.execution.kafka

import org.apache.gluten.execution.GlutenClickHouseWholeStageTransformerSuite

class ClickhouseGlutenKafkaScanSuite
extends GlutenClickHouseWholeStageTransformerSuite
with GlutenKafkaScanSuite {

override protected val fileFormat: String = "parquet"

protected val kafkaBootstrapServers: String = "localhost:9092"
}
Original file line number Diff line number Diff line change
Expand Up @@ -200,6 +200,7 @@ object CHBackendSettings extends BackendSettingsApi with Logging {
ValidationResult.failed("Has complex type.")
}
case JsonReadFormat => ValidationResult.succeeded
case KafkaReadFormat => ValidationResult.succeeded
case _ => ValidationResult.failed(s"Unsupported file format $format")
}
}
Expand All @@ -221,6 +222,7 @@ object CHBackendSettings extends BackendSettingsApi with Logging {
case "OrcScan" => ReadFileFormat.OrcReadFormat
case "ParquetScan" => ReadFileFormat.ParquetReadFormat
case "ClickHouseScan" => ReadFileFormat.MergeTreeReadFormat
case "KafkaScan" => ReadFileFormat.KafkaReadFormat
case _ => ReadFileFormat.UnknownFormat
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,8 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
(filesNode.toProtobuf.toByteArray, filesNode.getPaths.asScala.toSeq)
case extensionTableNode: ExtensionTableNode =>
(extensionTableNode.toProtobuf.toByteArray, extensionTableNode.getPartList)
case kafkaSourceNode: StreamKafkaSourceNode =>
(kafkaSourceNode.toProtobuf.toByteArray, Seq.empty)
}
}.unzip

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ class BatchScanMetricsUpdater(@transient val metrics: Map[String, SQLMetric])

object BatchScanMetricsUpdater {
// in mergetree format, the processor name is `MergeTreeSelect(pool: XXX, algorithm: XXX)`
val INCLUDING_PROCESSORS = Array("MergeTreeSelect(pool", "SubstraitFileSource")
val CH_PLAN_NODE_NAME = Array("MergeTreeSelect(pool", "SubstraitFileSource")
val INCLUDING_PROCESSORS =
Array("MergeTreeSelect(pool", "SubstraitFileSource", "GlutenKafkaSource")
val CH_PLAN_NODE_NAME = Array("MergeTreeSelect(pool", "SubstraitFileSource", "GlutenKafkaSource")
}
2 changes: 1 addition & 1 deletion cpp-ch/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,7 @@ else()
-DENABLE_TESTS=OFF -DENABLE_JEMALLOC=ON -DENABLE_MULTITARGET_CODE=ON
-DENABLE_EXTERN_LOCAL_ENGINE=ON -DENABLE_ODBC=OFF -DENABLE_CAPNP=OFF
-DENABLE_GRPC=OFF -DENABLE_RUST=OFF -DENABLE_H3=OFF -DENABLE_AMQPCPP=OFF
-DENABLE_CASSANDRA=OFF -DENABLE_KAFKA=OFF -DENABLE_NATS=OFF
-DENABLE_CASSANDRA=OFF -DENABLE_KAFKA=ON -DENABLE_NATS=OFF
-DENABLE_LIBPQXX=OFF -DENABLE_NURAFT=OFF -DENABLE_DATASKETCHES=OFF
-DENABLE_SQLITE=OFF -DENABLE_S2_GEOMETRY=OFF -DENABLE_ANNOY=OFF
-DENABLE_ULID=OFF -DENABLE_MYSQL=OFF -DENABLE_BCRYPT=OFF -DENABLE_LDAP=OFF
Expand Down
2 changes: 2 additions & 0 deletions cpp-ch/local-engine/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -53,10 +53,12 @@ add_headers_and_sources(parser Parser)
add_headers_and_sources(parser Parser/RelParsers)
add_headers_and_sources(rewriter Rewriter)
add_headers_and_sources(storages Storages)
add_headers_and_sources(storages Storages/Kafka)
add_headers_and_sources(storages Storages/Output)
add_headers_and_sources(storages Storages/Serializations)
add_headers_and_sources(storages Storages/IO)
add_headers_and_sources(storages Storages/MergeTree)
add_headers_and_sources(storages Storages/Kafka)
add_headers_and_sources(storages Storages/Cache)
add_headers_and_sources(common Common)
add_headers_and_sources(external External)
Expand Down
3 changes: 3 additions & 0 deletions cpp-ch/local-engine/Common/CHUtil.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
#include <DataTypes/NestedUtils.h>
#include <Disks/registerDisks.h>
#include <Disks/registerGlutenDisks.h>
#include <Formats/registerFormats.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/registerFunctions.h>
Expand Down Expand Up @@ -880,6 +881,8 @@ void registerGlutenDisks()

void BackendInitializerUtil::registerAllFactories()
{
registerFormats();

registerGlutenDisks();

registerReadBufferBuilders();
Expand Down
27 changes: 24 additions & 3 deletions cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
#include <Interpreters/Context.h>
#include <Operator/BlocksBufferPoolTransform.h>
#include <Parser/RelParsers/MergeTreeRelParser.h>
#include <Parser/RelParsers/StreamKafkaRelParser.h>
#include <Parser/SubstraitParserUtils.h>
#include <Parser/TypeParser.h>
#include <Processors/QueryPlan/ReadFromPreparedSource.h>
Expand All @@ -46,12 +47,12 @@ extern const int LOGICAL_ERROR;
namespace local_engine
{
using namespace DB;
DB::QueryPlanPtr ReadRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list<const substrait::Rel *> &)
DB::QueryPlanPtr ReadRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list<const substrait::Rel *> & rel_stack)
{
if (query_plan)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Source node's input plan should be null");
const auto & read = rel.read();
if (read.has_local_files() || (!read.has_extension_table() && !isReadFromMergeTree(read)))
if (isReadFromDefault(read))
{
assert(read.has_base_schema());
DB::QueryPlanStepPtr read_step;
Expand All @@ -70,6 +71,13 @@ DB::QueryPlanPtr ReadRelParser::parse(DB::QueryPlanPtr query_plan, const substra
query_plan->addStep(std::move(buffer_step));
}
}
else if (isReadFromStreamKafka(read))
{
StreamKafkaRelParser kafka_parser(parser_context, getContext());
kafka_parser.setSplitInfo(split_info);
query_plan = kafka_parser.parse(std::make_unique<DB::QueryPlan>(), rel, rel_stack);
steps = kafka_parser.getSteps();
}
else
{
substrait::ReadRel::ExtensionTable extension_table;
Expand All @@ -87,6 +95,11 @@ DB::QueryPlanPtr ReadRelParser::parse(DB::QueryPlanPtr query_plan, const substra
return query_plan;
}

bool ReadRelParser::isReadFromDefault(const substrait::ReadRel & read)
{
return read.has_local_files() || (!read.has_extension_table() && !isReadFromMergeTree(read) && !isReadFromStreamKafka(read));
}

bool ReadRelParser::isReadRelFromJava(const substrait::ReadRel & rel)
{
return rel.has_local_files() && rel.local_files().items().size() == 1
Expand All @@ -95,7 +108,9 @@ bool ReadRelParser::isReadRelFromJava(const substrait::ReadRel & rel)

bool ReadRelParser::isReadFromMergeTree(const substrait::ReadRel & rel)
{
assert(rel.has_advanced_extension());
if (!rel.has_advanced_extension())
return false;

bool is_read_from_merge_tree;
google::protobuf::StringValue optimization;
optimization.ParseFromString(rel.advanced_extension().optimization().value());
Expand All @@ -107,6 +122,12 @@ bool ReadRelParser::isReadFromMergeTree(const substrait::ReadRel & rel)
return is_read_from_merge_tree;
}


bool ReadRelParser::isReadFromStreamKafka(const substrait::ReadRel & rel)
{
return rel.has_stream_kafka() && rel.stream_kafka();
}

DB::QueryPlanStepPtr ReadRelParser::parseReadRelWithJavaIter(const substrait::ReadRel & rel)
{
GET_JNIENV(env)
Expand Down
4 changes: 3 additions & 1 deletion cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.h
Original file line number Diff line number Diff line change
Expand Up @@ -34,12 +34,14 @@ class ReadRelParser : public RelParser
return parse(std::move(query_plan), rel, rel_stack_);
}

DB::QueryPlanPtr parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list<const substrait::Rel *> &) override;
DB::QueryPlanPtr parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list<const substrait::Rel *> & rel_stack) override;
// This is source node, there is no input
std::optional<const substrait::Rel *> getSingleInput(const substrait::Rel & rel) override { return {}; }

bool isReadFromDefault(const substrait::ReadRel & rel);
bool isReadRelFromJava(const substrait::ReadRel & rel);
bool isReadFromMergeTree(const substrait::ReadRel & rel);
bool isReadFromStreamKafka(const substrait::ReadRel & rel);

void setInputIter(jobject input_iter_, bool is_materialze)
{
Expand Down
96 changes: 96 additions & 0 deletions cpp-ch/local-engine/Parser/RelParsers/StreamKafkaRelParser.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
/*
* 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.
*/

#include "StreamKafkaRelParser.h"

#include <Parser/SubstraitParserUtils.h>
#include <Parser/TypeParser.h>
#include <Storages/Kafka/ReadFromGlutenStorageKafka.h>

namespace DB
{

namespace ErrorCodes
{
extern const int NO_SUCH_DATA_PART;
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_FUNCTION;
extern const int UNKNOWN_TYPE;
}
}


namespace local_engine
{

DB::QueryPlanPtr
StreamKafkaRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list<const substrait::Rel *> & rel_stack_)
{
if (rel.has_read())
return parseRelImpl(std::move(query_plan), rel.read());

throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "StreamKafkaRelParser can't parse rel:{}", rel.ShortDebugString());
}

DB::QueryPlanPtr StreamKafkaRelParser::parseRelImpl(DB::QueryPlanPtr query_plan, const substrait::ReadRel & read_rel)
{
if (!read_rel.has_stream_kafka())
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Can't not parse kafka rel, because of read rel don't contained stream kafka");

auto kafka_task = BinaryToMessage<substrait::ReadRel::StreamKafka>(split_info);
auto topic = kafka_task.topic_partition().topic();
auto partition = kafka_task.topic_partition().partition();
auto start_offset = kafka_task.start_offset();
auto end_offset = kafka_task.end_offset();
auto poll_timeout_ms = kafka_task.poll_timeout_ms();
String group_id;
String brokers;

for (auto param : kafka_task.params())
if (param.first == "poll_timeout_ms")
poll_timeout_ms = std::stoi(param.second);
else if (param.first == "group.id")
group_id = param.second;
else if (param.first == "bootstrap.servers")
brokers = param.second;
else
LOG_DEBUG(getLogger("StreamKafkaRelParser"), "Unused kafka parameter: {}: {}", param.first, param.second);

LOG_INFO(
getLogger("StreamKafkaRelParser"),
"Kafka source: topic: {}, partition: {}, start_offset: {}, end_offset: {}",
topic,
partition,
start_offset,
end_offset);

Names topics;
topics.emplace_back(topic);

auto header = TypeParser::buildBlockFromNamedStruct(read_rel.base_schema());
Names names = header.getNames();
auto source = std::make_unique<ReadFromGlutenStorageKafka>(
names, header, getContext(), topics, partition, start_offset, end_offset, poll_timeout_ms, group_id, brokers);

steps.emplace_back(source.get());
query_plan->addStep(std::move(source));

return query_plan;
}


}
Loading