From d68f9b166746522ec3d80346d4ad15e489e3fb5b Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Sun, 28 Sep 2025 11:05:14 +0000 Subject: [PATCH 01/15] support proc time window --- .../stateful/InternalTimerService.h | 35 ++++++++++- .../stateful/ProcessingTimeService.h | 60 +++++++++++++++++++ .../stateful/StatefulPlanNode.cpp | 2 + .../experimental/stateful/StatefulPlanNode.h | 7 +++ .../experimental/stateful/StatefulPlanner.cpp | 7 ++- velox/experimental/stateful/Triggerable.h | 3 + .../experimental/stateful/WatermarkAssigner.h | 2 - .../stateful/WindowAggregator.cpp | 48 ++++++++++++--- .../experimental/stateful/WindowAggregator.h | 10 +++- 9 files changed, 158 insertions(+), 16 deletions(-) create mode 100644 velox/experimental/stateful/ProcessingTimeService.h diff --git a/velox/experimental/stateful/InternalTimerService.h b/velox/experimental/stateful/InternalTimerService.h index 362307b951d1..10fc11f27216 100644 --- a/velox/experimental/stateful/InternalTimerService.h +++ b/velox/experimental/stateful/InternalTimerService.h @@ -15,9 +15,12 @@ */ #pragma once +#include #include #include #include +#include +#include namespace facebook::velox::stateful { @@ -26,7 +29,7 @@ template class InternalTimerService { public: InternalTimerService(Triggerable* triggerable) - : triggerable_(triggerable) {} + : triggerable_(triggerable), processingTimeService_(std::make_shared()) {} void registerEventTimeTimer(K key, N ns, long time) { eventTimeTimersQueue_.add(std::make_shared>(time, key, ns)); @@ -37,6 +40,17 @@ class InternalTimerService { } void registerProcessingTimeTimer(K key, N ns, long time) { + std::shared_ptr> oldHead = processingTimeTimersQueue_.peek(); + processingTimeTimersQueue_.add(std::make_shared>(time, key, ns)); + long nextTriggerTime = oldHead != nullptr ? oldHead->timestamp() : std::numeric_limits::max() ; + if (time < nextTriggerTime) { + if (nextTimer_.hasValue() && !nextTimer_.isReady()) { + nextTimer_.cancel(); + } + nextTimer_ = processingTimeService_->registerTimer(time, [&](long time) { + onProcessingTime(time); + }); + } } void deleteProcessingTimeTimer(K key, N ns, long time) { @@ -69,8 +83,27 @@ class InternalTimerService { } private: + void onProcessingTime(long time) { + nextTimer_ = folly::Future::makeEmpty(); + std::shared_ptr> timer = nullptr; + while ((timer = processingTimeTimersQueue_.peek()) != nullptr + && timer->timestamp() <= time) { + processingTimeTimersQueue_.poll(); + triggerable_->onProcessingTime(timer); + } + + if (timer != nullptr && !nextTimer_.hasValue()) { + nextTimer_ = processingTimeService_->registerTimer(timer->timestamp(), [&](long time) { + onProcessingTime(time); + }); + } + } + Triggerable* triggerable_; + folly::Future nextTimer_; + std::shared_ptr processingTimeService_; HeapPriorityQueue>> eventTimeTimersQueue_; + HeapPriorityQueue>> processingTimeTimersQueue_; }; } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/ProcessingTimeService.h b/velox/experimental/stateful/ProcessingTimeService.h new file mode 100644 index 000000000000..aef7e3fff068 --- /dev/null +++ b/velox/experimental/stateful/ProcessingTimeService.h @@ -0,0 +1,60 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "velox/type/Timestamp.h" +#include +#include +#include + +namespace facebook::velox::stateful { + +using ProcessingTimeCallback = std::function; + +class ProcessingTimeSerivice { +public: + long getCurrentProcessingTime() { + return Timestamp::now().toMillis(); + } + + virtual folly::Future registerTimer(long timestamp, ProcessingTimeCallback target) { + return folly::Future::makeEmpty(); + } +}; + +class SystemProcessingTimeService : public ProcessingTimeSerivice { +public: + SystemProcessingTimeService() : ProcessingTimeSerivice() { + executor_ = std::make_shared(1); + } + + folly::Future registerTimer(long timestamp, ProcessingTimeCallback callback) override { + long currentTimestamp = getCurrentProcessingTime(); + long delay = 0; + if (timestamp >= currentTimestamp) { + delay = timestamp - currentTimestamp + 1; + } + return folly::futures::sleep(std::chrono::microseconds(delay * 1000)) + .via(executor_.get()) + .thenValue([&](auto) { + callback(timestamp); + }); + } +private: + std::shared_ptr executor_; + +}; +} \ No newline at end of file diff --git a/velox/experimental/stateful/StatefulPlanNode.cpp b/velox/experimental/stateful/StatefulPlanNode.cpp index 59a11c7c2690..8b96a7cb7259 100644 --- a/velox/experimental/stateful/StatefulPlanNode.cpp +++ b/velox/experimental/stateful/StatefulPlanNode.cpp @@ -248,6 +248,7 @@ folly::dynamic StreamWindowAggregationNode::serialize() const { obj["offset"] = offset_; obj["windowType"] = windowType_; obj["outputType"] = outputType_->serialize(); + obj["isEventTime"] = isEventTime_; obj["rowtimeIndex"] = rowtimeIndex_; return obj; } @@ -290,6 +291,7 @@ core::PlanNodePtr StreamWindowAggregationNode::create(const folly::dynamic& obj, obj["offset"].asInt(), obj["windowType"].asInt(), outputType, + obj["isEventTime"].asBool(), obj["rowtimeIndex"].asInt()); } diff --git a/velox/experimental/stateful/StatefulPlanNode.h b/velox/experimental/stateful/StatefulPlanNode.h index 0948bfadec76..e36fa4af5e24 100644 --- a/velox/experimental/stateful/StatefulPlanNode.h +++ b/velox/experimental/stateful/StatefulPlanNode.h @@ -333,6 +333,7 @@ class StreamWindowAggregationNode : public core::PlanNode { long offset, int windowType, const RowTypePtr& outputType, + bool isEventTime, int rowtimeIndex) : PlanNode(id), aggregation_(std::move(aggregationNode)), @@ -347,6 +348,7 @@ class StreamWindowAggregationNode : public core::PlanNode { offset_(offset), windowType_(windowType), outputType_(std::move(outputType)), + isEventTime_(isEventTime), rowtimeIndex_(rowtimeIndex) {} const RowTypePtr& outputType() const override { @@ -397,6 +399,10 @@ class StreamWindowAggregationNode : public core::PlanNode { return windowType_; } + bool isEventTime() const { + return isEventTime_; + } + int rowtimeIndex() const { return rowtimeIndex_; } @@ -426,6 +432,7 @@ class StreamWindowAggregationNode : public core::PlanNode { long offset_; int windowType_; const RowTypePtr outputType_; + bool isEventTime_; int rowtimeIndex_; }; diff --git a/velox/experimental/stateful/StatefulPlanner.cpp b/velox/experimental/stateful/StatefulPlanner.cpp index a0ab4366a0b2..fc4bce695ac7 100644 --- a/velox/experimental/stateful/StatefulPlanner.cpp +++ b/velox/experimental/stateful/StatefulPlanner.cpp @@ -172,7 +172,7 @@ StatefulOperatorPtr StatefulPlanner::nodeToStatefulOperator( windowAggNode->useDayLightSaving(), windowAggNode->outputType()); } else { - auto localAggregator = nodeToOperator(windowAggNode->localAgg(), ctx); + auto localAggregator = windowAggNode->isEventTime() ? nodeToOperator(windowAggNode->localAgg(), ctx) : nullptr; std::unique_ptr globalSliceAssigner = std::make_unique( std::move(sliceAssigner), @@ -182,13 +182,14 @@ StatefulOperatorPtr StatefulPlanner::nodeToStatefulOperator( windowAggNode->windowType(), windowAggNode->rowtimeIndex()); return std::make_unique( - std::move(localAggregator), + windowAggNode->isEventTime() ? std::move(localAggregator) : nullptr, std::move(op), std::move(targets), std::move(keySelector), std::move(globalSliceAssigner), windowAggNode->windowInterval(), - windowAggNode->useDayLightSaving()); + windowAggNode->useDayLightSaving(), + windowAggNode->isEventTime()); } } else if ( auto windowAggNode = diff --git a/velox/experimental/stateful/Triggerable.h b/velox/experimental/stateful/Triggerable.h index 0d3d1eb41dd4..e57b436cb93a 100644 --- a/velox/experimental/stateful/Triggerable.h +++ b/velox/experimental/stateful/Triggerable.h @@ -28,6 +28,9 @@ class Triggerable { public: virtual void onEventTime( std::shared_ptr> timer) = 0; + + virtual void onProcessingTime( + std::shared_ptr> timer) {} }; } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/WatermarkAssigner.h b/velox/experimental/stateful/WatermarkAssigner.h index 9623ab4a38df..e9478ee40655 100644 --- a/velox/experimental/stateful/WatermarkAssigner.h +++ b/velox/experimental/stateful/WatermarkAssigner.h @@ -15,9 +15,7 @@ */ #pragma once -#include "velox/exec/FilterProject.h" #include "velox/experimental/stateful/StatefulOperator.h" -#include "velox/experimental/stateful/StatefulPlanNode.h" namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 99a14ef65ba0..0c8cca9f2715 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -14,6 +14,7 @@ * limitations under the License. */ #include "velox/experimental/stateful/WindowAggregator.h" +#include #include "velox/experimental/stateful/window/TimeWindowUtil.h" #include @@ -27,13 +28,15 @@ WindowAggregator::WindowAggregator( std::unique_ptr keySelector, std::unique_ptr sliceAssigner, const long windowInterval, - const bool useDayLightSaving) + const bool useDayLightSaving, + const bool isEventTime) : StatefulOperator(std::move(globalAggerator), std::move(targets)), localAggerator_(std::move(localAggerator)), keySelector_(std::move(keySelector)), sliceAssigner_(std::move(sliceAssigner)), windowInterval_(windowInterval), - useDayLightSaving_(useDayLightSaving) { + useDayLightSaving_(useDayLightSaving), + isEventTime_(isEventTime) { windowBuffer_ = std::make_shared(); } @@ -62,12 +65,11 @@ void WindowAggregator::getOutput() { sliceAssigner_->assignSliceEnd(data); for (const auto& [sliceEnd, data] : sliceEndToData) { auto windowData = data; - if (!isEventTime) { - // TODO: support processing time - //windowTimerService_->registerProcessingTimeWindowTimer(sliceEnd, sliceEnd - 1); + if (!isEventTime_) { + windowTimerService_->registerProcessingTimeTimer(key, sliceEnd, sliceEnd - 1); } - if (isEventTime && TimeWindowUtil::isWindowFired(sliceEnd, currentProgress_, shiftTimeZone_)) { + if (isEventTime_ && TimeWindowUtil::isWindowFired(sliceEnd, currentProgress_, shiftTimeZone_)) { // the assigned slice has been triggered, which means current element is late, // but maybe not need to drop long lastWindowEnd = sliceAssigner_->getLastWindowEnd(sliceEnd); @@ -97,7 +99,7 @@ void WindowAggregator::getOutput() { } void WindowAggregator::processWatermarkInternal(long timestamp) { - if (isEventTime && timestamp > currentProgress_) { + if (isEventTime_ && timestamp > currentProgress_) { currentProgress_ = timestamp; if (currentProgress_ >= nextTriggerWatermark_) { // we only need to call advanceProgress() when current watermark may trigger window @@ -135,12 +137,42 @@ void WindowAggregator::processWatermarkInternal(long timestamp) { } } -void WindowAggregator::onEventTime(std::shared_ptr> timer) { +void WindowAggregator::onTimer(std::shared_ptr> timer) { auto output = windowState_->value(timer->key(), timer->ns()); windowState_->remove(timer->key(), timer->ns()); pushOutput(output); } +void WindowAggregator::onEventTime(std::shared_ptr> timer) { + onTimer(timer); +} + +void WindowAggregator::onProcessingTime(std::shared_ptr> timer) { + if (timer->timestamp() > lastTriggeredProcessingTime_) { + lastTriggeredProcessingTime_ = timer->timestamp(); + auto windowKeyToData = windowBuffer_->advanceProgress(timer->timestamp()); + for (const auto&[windowKey, datas] : windowKeyToData) { + if (datas.empty()) { + continue; + } + std::list allDatas; + for (const auto& data: datas) { + allDatas.push_back(data); + } + auto stateAcc = windowState_->value(windowKey.key(), windowKey.window()); + if (stateAcc) { + allDatas.push_back(stateAcc); + } + op()->addInput(TimeWindowUtil::mergeVectors(allDatas, op()->pool())); + auto newAcc = op()->getOutput(); + if (newAcc) { + windowState_->update(windowKey.key(), windowKey.window(), newAcc); + } + } + onTimer(timer); + } +} + long WindowAggregator::sliceStateMergeTarget(long sliceToMerge) { // TODO: implement it return sliceToMerge; diff --git a/velox/experimental/stateful/WindowAggregator.h b/velox/experimental/stateful/WindowAggregator.h index 28ed705b0abc..72ce565b3fc2 100644 --- a/velox/experimental/stateful/WindowAggregator.h +++ b/velox/experimental/stateful/WindowAggregator.h @@ -36,7 +36,8 @@ class WindowAggregator : public StatefulOperator, public Triggerable keySelector, std::unique_ptr sliceAssigner, const long windowInterval, - const bool useDayLightSaving); + const bool useDayLightSaving, + const bool isEventTime); void initialize() override; @@ -52,11 +53,15 @@ class WindowAggregator : public StatefulOperator, public Triggerable> timer) override; + void onProcessingTime(std::shared_ptr> timer) override; + private: void processWatermarkInternal(long timestamp) override; long sliceStateMergeTarget(long sliceToMerge); + void onTimer(std::shared_ptr> timer); + std::unique_ptr localAggerator_; std::unique_ptr keySelector_; std::unique_ptr sliceAssigner_; @@ -64,11 +69,12 @@ class WindowAggregator : public StatefulOperator, public Triggerable> windowState_; std::shared_ptr> windowTimerService_; }; From 4a98cee6367f6dc33ae5d98694059e02b1223f53 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Tue, 14 Oct 2025 07:41:04 +0000 Subject: [PATCH 02/15] support proctime window --- CMakeLists.txt | 7 + velox/connectors/CMakeLists.txt | 1 + velox/connectors/fuzzer/DiscardDataSink.cpp | 1 + velox/connectors/kafka/CMakeLists.txt | 40 ++++ velox/connectors/kafka/KafkaConfig.cpp | 111 +++++++++++ velox/connectors/kafka/KafkaConfig.h | 146 ++++++++++++++ velox/connectors/kafka/KafkaConnector.cpp | 42 ++++ velox/connectors/kafka/KafkaConnector.h | 81 ++++++++ .../connectors/kafka/KafkaConnectorSplit.cpp | 113 +++++++++++ velox/connectors/kafka/KafkaConnectorSplit.h | 81 ++++++++ velox/connectors/kafka/KafkaConsumer.cpp | 111 +++++++++++ velox/connectors/kafka/KafkaConsumer.h | 70 +++++++ velox/connectors/kafka/KafkaDataSource.cpp | 165 ++++++++++++++++ velox/connectors/kafka/KafkaDataSource.h | 118 +++++++++++ velox/connectors/kafka/KafkaTableHandle.cpp | 86 ++++++++ velox/connectors/kafka/KafkaTableHandle.h | 76 ++++++++ velox/connectors/kafka/format/CMakeLists.txt | 21 ++ .../kafka/format/CSVRecordDeserializer.h | 37 ++++ .../kafka/format/JSONRecordDeserializer.cpp | 103 ++++++++++ .../kafka/format/JSONRecordDeserializer.h | 160 +++++++++++++++ .../kafka/format/KafkaRecordDeserializer.h | 55 ++++++ .../kafka/format/RawRecordDeserializer.h | 56 ++++++ .../format/StreamJSONRecordDeserializer.cpp | 71 +++++++ .../format/StreamJSONRecordDeserializer.h | 165 ++++++++++++++++ velox/connectors/kafka/tests/CMakeLists.txt | 27 +++ .../kafka/tests/KafkaConnectorTest.cpp | 163 ++++++++++++++++ .../kafka/tests/KafkaConnectorTestBase.h | 184 ++++++++++++++++++ velox/connectors/kafka/tests/kafka.conf | 2 + .../stateful/GroupWindowAggregator.cpp | 4 +- .../stateful/InternalPriorityQueue.h | 6 + .../stateful/InternalTimerService.h | 46 +++-- velox/experimental/stateful/KeySelector.cpp | 8 +- velox/experimental/stateful/KeySelector.h | 2 +- .../stateful/LocalWindowAggregator.cpp | 4 +- .../stateful/ProcessingTimeService.h | 81 ++++++-- .../stateful/StatefulPlanNode.cpp | 6 +- .../experimental/stateful/StatefulPlanNode.h | 18 +- .../experimental/stateful/StatefulPlanner.cpp | 6 +- .../stateful/WindowAggregator.cpp | 124 ++++++++++-- .../experimental/stateful/WindowAggregator.h | 19 +- .../stateful/window/MergingWindowSet.h | 1 - .../stateful/window/SliceAssigner.cpp | 17 +- .../stateful/window/SliceAssigner.h | 2 +- .../stateful/window/TimeWindowUtil.cpp | 5 + .../stateful/window/TimeWindowUtil.h | 2 + 45 files changed, 2581 insertions(+), 63 deletions(-) create mode 100644 velox/connectors/kafka/CMakeLists.txt create mode 100644 velox/connectors/kafka/KafkaConfig.cpp create mode 100644 velox/connectors/kafka/KafkaConfig.h create mode 100644 velox/connectors/kafka/KafkaConnector.cpp create mode 100644 velox/connectors/kafka/KafkaConnector.h create mode 100644 velox/connectors/kafka/KafkaConnectorSplit.cpp create mode 100644 velox/connectors/kafka/KafkaConnectorSplit.h create mode 100644 velox/connectors/kafka/KafkaConsumer.cpp create mode 100644 velox/connectors/kafka/KafkaConsumer.h create mode 100644 velox/connectors/kafka/KafkaDataSource.cpp create mode 100644 velox/connectors/kafka/KafkaDataSource.h create mode 100644 velox/connectors/kafka/KafkaTableHandle.cpp create mode 100644 velox/connectors/kafka/KafkaTableHandle.h create mode 100644 velox/connectors/kafka/format/CMakeLists.txt create mode 100644 velox/connectors/kafka/format/CSVRecordDeserializer.h create mode 100644 velox/connectors/kafka/format/JSONRecordDeserializer.cpp create mode 100644 velox/connectors/kafka/format/JSONRecordDeserializer.h create mode 100644 velox/connectors/kafka/format/KafkaRecordDeserializer.h create mode 100644 velox/connectors/kafka/format/RawRecordDeserializer.h create mode 100644 velox/connectors/kafka/format/StreamJSONRecordDeserializer.cpp create mode 100644 velox/connectors/kafka/format/StreamJSONRecordDeserializer.h create mode 100644 velox/connectors/kafka/tests/CMakeLists.txt create mode 100644 velox/connectors/kafka/tests/KafkaConnectorTest.cpp create mode 100644 velox/connectors/kafka/tests/KafkaConnectorTestBase.h create mode 100644 velox/connectors/kafka/tests/kafka.conf diff --git a/CMakeLists.txt b/CMakeLists.txt index 45fd5bb56a1e..cbf6282ad595 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -128,6 +128,7 @@ set(VELOX_GFLAGS_TYPE option(VELOX_ENABLE_EXEC "Build exec." ON) option(VELOX_ENABLE_AGGREGATES "Build aggregates." ON) option(VELOX_ENABLE_HIVE_CONNECTOR "Build Hive connector." ON) +option(VELOX_ENABLE_KAFKA_CONNECTOR "Build Kafka connector." ON) option(VELOX_ENABLE_TPCH_CONNECTOR "Build TPC-H connector." ON) option(VELOX_ENABLE_PRESTO_FUNCTIONS "Build Presto SQL functions." ON) option(VELOX_ENABLE_SPARK_FUNCTIONS "Build Spark SQL functions." ON) @@ -175,6 +176,7 @@ if(${VELOX_BUILD_MINIMAL} OR ${VELOX_BUILD_MINIMAL_WITH_DWIO}) set(VELOX_ENABLE_EXEC OFF) set(VELOX_ENABLE_AGGREGATES OFF) set(VELOX_ENABLE_HIVE_CONNECTOR OFF) + set(VELOX_ENABLE_KAFKA_CONNECTOR ON) set(VELOX_ENABLE_TPCH_CONNECTOR OFF) set(VELOX_ENABLE_SPARK_FUNCTIONS OFF) set(VELOX_ENABLE_EXAMPLES OFF) @@ -188,6 +190,11 @@ if(${VELOX_ENABLE_BENCHMARKS}) set(VELOX_ENABLE_BENCHMARKS_BASIC ON) endif() +if(${VELOX_ENABLE_KAFKA_CONNECTOR}) + velox_set_source(CppKafka) + velox_resolve_dependency(CppKafka) +endif() + if(VELOX_ENABLE_BENCHMARKS_BASIC) set(VELOX_BUILD_TEST_UTILS ON) endif() diff --git a/velox/connectors/CMakeLists.txt b/velox/connectors/CMakeLists.txt index 3c2c758cc41a..032097691161 100644 --- a/velox/connectors/CMakeLists.txt +++ b/velox/connectors/CMakeLists.txt @@ -20,6 +20,7 @@ add_subdirectory(nexmark) add_subdirectory(print) add_subdirectory(from_elements) add_subdirectory(utils) +add_subdirectory(kafka) if(${VELOX_ENABLE_HIVE_CONNECTOR}) add_subdirectory(hive) diff --git a/velox/connectors/fuzzer/DiscardDataSink.cpp b/velox/connectors/fuzzer/DiscardDataSink.cpp index 7a11d060b646..40ab880f9ca7 100644 --- a/velox/connectors/fuzzer/DiscardDataSink.cpp +++ b/velox/connectors/fuzzer/DiscardDataSink.cpp @@ -32,6 +32,7 @@ void DiscardDataSink::appendData(RowVectorPtr input) { if (lastTime == 0) { lastTime = getCurrentTime(); } + LOG(INFO) << "input:" << input->toString(0); auto preNum = rowNums; rowNums += input->size(); if (rowNums / 100000 != preNum / 100000) { diff --git a/velox/connectors/kafka/CMakeLists.txt b/velox/connectors/kafka/CMakeLists.txt new file mode 100644 index 000000000000..24f61712c4d4 --- /dev/null +++ b/velox/connectors/kafka/CMakeLists.txt @@ -0,0 +1,40 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed 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. + +# velox_add_library(velox_kafka_config OBJECT KafkaConfig.cpp) +# velox_link_libraries(velox_kafka_config velox_core velox_exception) + +add_subdirectory(format) + +velox_add_library( + velox_kafka_connector + OBJECT + KafkaConfig.cpp + KafkaConsumer.cpp + KafkaTableHandle.cpp + KafkaDataSource.cpp + KafkaConnector.cpp + KafkaConnectorSplit.cpp) + +velox_link_libraries( + velox_kafka_connector + velox_kafka_connector_format + velox_common_io + velox_connector + Folly::folly + CppKafka::cppkafka) + +if(${VELOX_BUILD_TESTING}) + add_subdirectory(tests) +endif() diff --git a/velox/connectors/kafka/KafkaConfig.cpp b/velox/connectors/kafka/KafkaConfig.cpp new file mode 100644 index 000000000000..16ec3df6eebf --- /dev/null +++ b/velox/connectors/kafka/KafkaConfig.cpp @@ -0,0 +1,111 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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 "velox/connectors/kafka/KafkaConfig.h" + +namespace facebook::velox::connector::kafka { + +template +const T KafkaConfig::checkAndGetConfigValue( + const std::string& configKey, + T defaultValue) const { + std::optional configValue = + static_cast>(config_->get(configKey)); + if constexpr (throwException) { + VELOX_CHECK_EQ( + configValue.has_value(), + true, + "Kafka config {} has no specified value.", + configKey); + } + if (configValue.has_value()) { + return configValue.value(); + } else { + return defaultValue; + } +} + +const std::string ConnectionConfig::getBootstrapServers() const { + return checkAndGetConfigValue(kBootstrapServers, ""); +} + +const std::string ConnectionConfig::getTopic() const { + return checkAndGetConfigValue(kTopic, ""); +} + +const std::string ConnectionConfig::getGroupId() const { + return checkAndGetConfigValue(kGroupId, ""); +} + +const std::string ConnectionConfig::getClientId() const { + return checkAndGetConfigValue(kClientId, ""); +} + +const std::string ConnectionConfig::getFormat() const { + return checkAndGetConfigValue(kFormat, ""); +} + +const std::string ConnectionConfig::getAutoOffsetReset() const { + return checkAndGetConfigValue( + kAutoResetOffset, defaultAutoOffsetRest); +} + +const uint32_t ConnectionConfig::getQueuedMinMessages() const { + return checkAndGetConfigValue( + kQueueMinMessages, defaultQueuedMinMessages); +} + +const bool ConnectionConfig::getEnableAutoCommit() const { + return checkAndGetConfigValue(kEnableAutoCommit, "true") == + "true" + ? true + : false; +} + +const bool ConnectionConfig::getEnablePartitionEof() const { + return checkAndGetConfigValue(kEnablePartitionEof, false); +} + +const uint32_t ConnectionConfig::getDataBatchSize() const { + return checkAndGetConfigValue( + kDataBatchSize, defaultDataBatchSize); +} + +const uint32_t ConnectionConfig::getPollTimeoutMills() const { + return checkAndGetConfigValue( + kPollTimeoutMills, defaultPollTimeoutMills); +} + +const std::string ConnectionConfig::getStartupMode() const { + return checkAndGetConfigValue( + kStartupMode, defaultConsumeStartupMode); +} + +cppkafka::Configuration ConnectionConfig::getCppKafkaConfiguration() const { + cppkafka::Configuration conf; + conf.set("metadata.broker.list", getBootstrapServers()); + conf.set("group.id", getGroupId()); + conf.set("client.id", getClientId()); + conf.set("client.software.name", defaultClientSoftwareName); + conf.set("client.software.version", defaultClientSoftwareVersion); + conf.set("auto.offset.reset", getAutoOffsetReset()); + conf.set("queued.min.messages", getQueuedMinMessages()); + conf.set("enable.auto.commit", getEnableAutoCommit()); + conf.set("auto.commit.interval.ms", 2000); + conf.set("enable.partition.eof", getEnablePartitionEof()); + return conf; +} +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConfig.h b/velox/connectors/kafka/KafkaConfig.h new file mode 100644 index 000000000000..2a2009882b66 --- /dev/null +++ b/velox/connectors/kafka/KafkaConfig.h @@ -0,0 +1,146 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "velox/common/config/Config.h" +#include + +namespace facebook::velox::connector::kafka { + +using ConfigPtr = std::shared_ptr; + +/// Kafka config base class. +class KafkaConfig { + public: + KafkaConfig(const ConfigPtr& config) : config_(config) {} + + const ConfigPtr& getConfig() const { + return config_; + } + + const bool exists(const std::string& configKey) const { + return config_ && config_->valueExists(configKey); + } + + const bool empty() const { + if (!config_) { + return true; + } else { + return config_->rawConfigs().empty(); + } + } + + template + const std::shared_ptr setConfigs( + const std::unordered_map& configs) const { + std::unordered_map rawConfigs = config_->rawConfigsCopy(); + rawConfigs.insert(configs.begin(), configs.end()); + ConfigPtr newConfig = + std::make_shared(std::move(rawConfigs)); + return std::make_shared(newConfig); + } + + protected: + ConfigPtr config_; + template + const T checkAndGetConfigValue(const std::string& configKey, T defaultValue) const; +}; + +/// Kafka connector config. +class ConnectionConfig : public KafkaConfig { + public: + ConnectionConfig(const ConfigPtr& config) : KafkaConfig(config) {} + /// The config key of bootstrap servers + static constexpr const char* kBootstrapServers = "bootstrap.servers"; + /// The config key of topic + static constexpr const char* kTopic = "topic"; + /// The config key of group id + static constexpr const char* kGroupId = "group.id"; + /// The config key of client id + static constexpr const char* kClientId = "client.id"; + /// The config key fo format + static constexpr const char* kFormat = "format"; + /// The config key of auto offset reset + static constexpr const char* kAutoResetOffset = "auto.offset.reset"; + /// The config key of minimum number of messages of the queue buffer + static constexpr const char* kQueueMinMessages = "queued.min.messages"; + /// The config key of whether to enable auto commit kafka coffset + static constexpr const char* kEnableAutoCommit = "enable.auto.commit"; + /// The config key of whether to ignore partition eof + static constexpr const char* kEnablePartitionEof = "enable.partition.eof"; + /// The config key of max batch size to poll kafka messages. + static constexpr const char* kDataBatchSize = "data.batch.size"; + /// The config key of timeout milliseconds to poll kafka messages. + static constexpr const char* kPollTimeoutMills = "poll.timeout.mills"; + /// The config key of queue buffer size of cppkafka client + static constexpr const char* kConsumeMessageQueueSize = "consume.queue.size"; + /// The startup mode of kafka consumer, its value canbe `group-offsets`, + /// `latest-offsets`, `earliest-offsets`, `timestamp`. + static constexpr const char* kStartupMode = "scan.startup.mode"; + /// The config of kafka client, to define the default value of minimum + /// messages size of kafka client queue. + static constexpr const uint32_t defaultQueuedMinMessages = 1000000; + /// The config of the kafka client, to define the default software name of + /// client. + static constexpr const char* defaultClientSoftwareName = "velox"; + /// The config of the kafka client, to define the default version of client. + static constexpr const char* defaultClientSoftwareVersion = "***"; + /// Define the default batch size of a data process. + static constexpr const uint32_t defaultDataBatchSize = 500; + /// Define the default poll batch size of kafka client. + static constexpr const uint32_t defaultPollBatchSize = 500; + /// The config of the kafka client, to define the default timeout millseconds + /// of a single consumption. + static constexpr const uint32_t defaultPollTimeoutMills = 100; + /// The config of kafka client, to define the default value of config + /// `auto.offset.reset` + static constexpr const char* defaultAutoOffsetRest = "latest"; + /// The config of kafka client, to define the default value of config + /// `scan.startup.mode` + static constexpr const char* defaultConsumeStartupMode = "group-offsets"; + + const std::string getBootstrapServers() const; + const std::string getTopic() const; + const std::string getGroupId() const; + const std::string getClientId() const; + const std::string getFormat() const; + const std::string getAutoOffsetReset() const; + const uint32_t getQueuedMinMessages() const; + const bool getEnableAutoCommit() const; + const bool getEnablePartitionEof() const; + const uint32_t getDataBatchSize() const; + const uint32_t getPollTimeoutMills() const; + const uint32_t getConsumeQueueSize() const; + const std::string getStartupMode() const; + /// Get the configuration for kafka client to consume. + cppkafka::Configuration getCppKafkaConfiguration() const; +}; + +/// The json format config for kafka. +class JSONFormatConfig : public KafkaConfig { + public: + JSONFormatConfig(const ConfigPtr& config) : KafkaConfig(config) {} +}; + +/// The csv format config for kafka. +class CSVFormatConfig : public KafkaConfig {}; + +using KafkaConfigPtr = std::shared_ptr; +using ConnectionConfigPtr = std::shared_ptr; +using JSONFormatConfigPtr = std::shared_ptr; +using CSVFormatConfigPtr = std::shared_ptr; + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConnector.cpp b/velox/connectors/kafka/KafkaConnector.cpp new file mode 100644 index 000000000000..71b92dd434ea --- /dev/null +++ b/velox/connectors/kafka/KafkaConnector.cpp @@ -0,0 +1,42 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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 "velox/connectors/kafka/KafkaConnector.h" +#include "velox/connectors/kafka/KafkaDataSource.h" + +namespace facebook::velox::connector::kafka { + +std::unique_ptr KafkaConnector::createDataSource( + const RowTypePtr& outputType, + const ConnectorHandlePtr& tableHandle, + const std::unordered_map>& /* columnHandles **/, + ConnectorQueryCtx* connectorQueryCtx) { + return std::make_unique( + outputType, + tableHandle, + connectorQueryCtx, + config_); +} + +std::unique_ptr KafkaConnector::createDataSink( + RowTypePtr inputType, + ConnectorInsertTableHandlePtr connectorInsertTableHandle, + ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy) { + VELOX_NYI(); +} + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConnector.h b/velox/connectors/kafka/KafkaConnector.h new file mode 100644 index 000000000000..04a1f22aadca --- /dev/null +++ b/velox/connectors/kafka/KafkaConnector.h @@ -0,0 +1,81 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/connectors/kafka/KafkaConfig.h" +#include "velox/connectors/kafka/KafkaDataSource.h" + +namespace facebook::velox::connector::kafka { + +using ConnectorHandlePtr = std::shared_ptr; +using ConnectorInsertTableHandlePtr = + std::shared_ptr; + +/// The kafka connector. +class KafkaConnector : public Connector { + public: + KafkaConnector( + const std::string& id, + std::shared_ptr config, + folly::Executor* /* executor **/) + : Connector(id), + config_(std::make_shared(config)) {} + + const std::shared_ptr& connectorConfig() + const override { + return config_->getConfig(); + } + + ConnectorMetadata* metadata() const override { + VELOX_NYI(); + } + + std::unique_ptr createDataSource( + const RowTypePtr& outputType, + const ConnectorHandlePtr& tableHandle, + const std::unordered_map>& columnHandles, + ConnectorQueryCtx* connectorQueryCtx) override; + + std::unique_ptr createDataSink( + RowTypePtr inputType, + ConnectorInsertTableHandlePtr connectorInsertTableHandle, + ConnectorQueryCtx* connectorQueryCtx, + CommitStrategy commitStrategy) override; + + private: + const ConnectionConfigPtr config_; +}; + +class KafkaConnectorFactory : public ConnectorFactory { + public: + static constexpr const char* kKafkaConnectorName{"Kafka"}; + + KafkaConnectorFactory() : ConnectorFactory(kKafkaConnectorName) {} + + explicit KafkaConnectorFactory(const char* connectorName) + : ConnectorFactory(connectorName) {} + + std::shared_ptr newConnector( + const std::string& id, + std::shared_ptr config, + folly::Executor* ioExecutor = nullptr, + folly::Executor* cpuExecutor = nullptr) override { + return std::make_shared(id, config, ioExecutor); + } +}; + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConnectorSplit.cpp b/velox/connectors/kafka/KafkaConnectorSplit.cpp new file mode 100644 index 000000000000..14a5d3b8adc3 --- /dev/null +++ b/velox/connectors/kafka/KafkaConnectorSplit.cpp @@ -0,0 +1,113 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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 "velox/connectors/kafka/KafkaConnectorSplit.h" +#include +#include + +namespace facebook::velox::connector::kafka { + +std::string KafkaConnectorSplit::topicPartitonsToString( + const cppkafka::TopicPartitionList& tps) { + if (tps.empty()) { + return ""; + } + std::stringstream s; + for (size_t i = 0; i < tps.size() - 1; ++i) { + s << "[" << tps[i].get_topic() << "," << tps[i].get_partition() << "]"; + s << ","; + } + s << "[" << tps[tps.size() - 1].get_topic() << "," + << tps[tps.size() - 1].get_partition() << "]"; + return s.str(); +} + +std::string KafkaConnectorSplit::toString() const { + return fmt::format( + "Kafka connector split, connectorId: {}, bootstrap servers:{}, topic partitions:{}, group id:{}", + clientId_, + bootstrapServers_, + KafkaConnectorSplit::topicPartitonsToString(getCppKafkaTopicPartitions()), + groupId_); +} + +folly::dynamic KafkaConnectorSplit::serialize() const { + folly::dynamic obj = folly::dynamic::object; + obj["connectorId"] = clientId_; + obj["bootstrapServers"] = bootstrapServers_; + obj["groupId"] = groupId_; + obj["format"] = format_; + obj["enableAutoCommit"] = enableAutoCommit_; + obj["autoResetOffset"] = autoResetOffset_; + folly::dynamic tpObjs = folly::dynamic::array; + size_t j = 0; + for (const auto& tp : topicPartitions_) { + std::string topic = tp.first; + for (size_t i = 0; i < tp.second.size(); ++i) { + folly::dynamic d = folly::dynamic::object; + d["topic"] = topic; + d["partition"] = tp.second[i].first; + d["offset"] = tp.second[i].second; + tpObjs[j] = d; + j++; + } + } + obj["topicPartitions"] = tpObjs; + return obj; +} + +std::shared_ptr KafkaConnectorSplit::create( + const folly::dynamic& obj) { + std::unordered_map>> topics; + if (obj["topicPartitions"].isArray()) { + const auto tpObjs = obj["topicPartitions"]; + for (const auto& tp : tpObjs) { + if (tp.isObject()) { + const auto topic = tp["topic"]; + const auto partition = tp["partition"]; + const auto offset = tp["offset"]; + const auto it = topics.find(topic.asString()); + std::pair p( + static_cast(partition.asInt()), + static_cast(offset.asInt())); + if (it != topics.end()) { + std::vector>& partitionOffset = + it->second; + partitionOffset.emplace_back(p); + } else { + std::vector> partitionOffset; + partitionOffset.emplace_back(p); + topics[topic.asString()] = partitionOffset; + } + } + } + } + return std::make_shared( + obj["connectorId"].asString(), + obj["bootstrapServers"].asString(), + obj["groupId"].asString(), + obj["format"].asString(), + obj["enableAutoCommit"].asBool(), + obj["autoResetOffset"].asString(), + topics); +} + +void KafkaConnectorSplit::registerSerDe() { + auto& registry = DeserializationRegistryForSharedPtr(); + registry.Register("KafkaConnectorSplit", KafkaConnectorSplit::create); +} + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConnectorSplit.h b/velox/connectors/kafka/KafkaConnectorSplit.h new file mode 100644 index 000000000000..9bf570ec0977 --- /dev/null +++ b/velox/connectors/kafka/KafkaConnectorSplit.h @@ -0,0 +1,81 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "velox/connectors/Connector.h" +#include +#include +#include + +namespace facebook::velox::connector::kafka { + +/// The split information for kafka connector. +struct KafkaConnectorSplit : public ConnectorSplit { + std::string bootstrapServers_; + std::string groupId_; + std::string clientId_; + std::string format_; + bool enableAutoCommit_; + std::string autoResetOffset_; + std::unordered_map>> + topicPartitions_; + + explicit KafkaConnectorSplit( + const std::string& connectorId, + const std::string& bootstrapServers, + const std::string& groupId, + const std::string& format, + const bool enableAutoCommit, + const std::string& autoResetOffset, + std::unordered_map>>& + tps) + : ConnectorSplit(connectorId), + bootstrapServers_(bootstrapServers), + groupId_(groupId), + clientId_(connectorId), + format_(format), + enableAutoCommit_(enableAutoCommit), + autoResetOffset_(autoResetOffset), + topicPartitions_(tps) {} + + cppkafka::TopicPartitionList getCppKafkaTopicPartitions() const { + cppkafka::TopicPartitionList topicPartitions; + for (const auto& p : topicPartitions_) { + std::string topic = p.first; + for (const auto& partition : p.second) { + cppkafka::TopicPartition topicPartition( + topic, static_cast(partition.first)); + if (partition.second >= 0) { + topicPartition.set_offset(partition.second); + } + topicPartitions.emplace_back(topicPartition); + } + } + return topicPartitions; + } + + static std::string topicPartitonsToString(const cppkafka::TopicPartitionList& tps); + + std::string toString() const override; + + folly::dynamic serialize() const override; + + static std::shared_ptr create(const folly::dynamic& obj); + + static void registerSerDe(); +}; +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConsumer.cpp b/velox/connectors/kafka/KafkaConsumer.cpp new file mode 100644 index 000000000000..54035213f93f --- /dev/null +++ b/velox/connectors/kafka/KafkaConsumer.cpp @@ -0,0 +1,111 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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 "velox/connectors/kafka/KafkaConsumer.h" +#include "velox/connectors/kafka/KafkaConnectorSplit.h" +#include +#include +#include + +namespace facebook::velox::connector::kafka { + +void KafkaConsumer::subscribe(const std::vector& topics) { + auto topicsToString = [&]() -> std::string { + if (topics.empty()) { + return ""; + } + std::stringstream s; + for (size_t i = 0; i < topics.size() - 1; ++i) { + s << topics[i]; + s << ","; + } + s << topics[topics.size() - 1]; + return s.str(); + }; + VELOX_CHECK_NOT_NULL( + consumer_.get(), + "Failed to subscribe to topics: {}, as the cppkafka consumer is null.", + topicsToString()); + consumer_->subscribe(topics); +} + +const cppkafka::TopicPartitionList KafkaConsumer::getTopicPartitions( + const std::string& topic, + const std::string& startupMode) { + cppkafka::TopicPartitionList tps; + auto metadata = consumer_->get_metadata(); + const auto& topics = metadata.get_topics(); + for (const cppkafka::TopicMetadata& topicMetadata : topics) { + if (topicMetadata.get_name() == topic) { + const auto& partitions = topicMetadata.get_partitions(); + for (const auto& partition : partitions) { + cppkafka::TopicPartition topicPartition( + topic, static_cast(partition.get_id())); + auto offsets = consumer_->query_offsets(topicPartition); + if (startupMode == "earliest-offsets") { + topicPartition.set_offset(std::get<0>(offsets)); + } else if (startupMode == "latest-offsets") { + topicPartition.set_offset(std::get<1>(offsets)); + } + tps.emplace_back(topicPartition); + } + } + } + VELOX_CHECK_GT(tps.size(), 0, "Failed to get partitions of topic:{}", topic); + return tps; +} + +const void KafkaConsumer::setTopicPartitionsOffset(cppkafka::TopicPartitionList& tps, const std::string& startupMode) { + for (cppkafka::TopicPartition & tp : tps) { + auto offsets = consumer_->query_offsets(tp); + if (startupMode == "earliest-offsets") { + tp.set_offset(std::get<0>(offsets)); + } else if (startupMode == "latest-offsets") { + tp.set_offset(std::get<1>(offsets)); + } + } +} + +void KafkaConsumer::assign(const cppkafka::TopicPartitionList& tps) { + std::string tpsString = KafkaConnectorSplit::topicPartitonsToString(tps); + VELOX_CHECK_NOT_NULL( + consumer_.get(), + "Failed to assign topic partitions: {}, as the cppkafka consumer is null.", + tpsString); + consumer_->assign(tps); +} + +const void KafkaConsumer::consumeBatch( + std::vector& res, + size_t& msgBytes) { + const std::vector msgs = + consumer_->poll_batch(pollBatchSize_); + for (const auto& msg : msgs) { + const std::string& msgData = msg.get_payload(); + msgBytes += msgData.size(); + res.emplace_back(msgData); + } +} + +const std::vector KafkaConsumer::getSubscribedTopics() { + return consumer_->get_subscription(); +} + +const cppkafka::TopicPartitionList KafkaConsumer::getAssignedTopicPartitions() { + return consumer_->get_assignment(); +} + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConsumer.h b/velox/connectors/kafka/KafkaConsumer.h new file mode 100644 index 000000000000..b27a0d76c79f --- /dev/null +++ b/velox/connectors/kafka/KafkaConsumer.h @@ -0,0 +1,70 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "velox/connectors/kafka/KafkaConfig.h" +#include +#include + +namespace facebook::velox::connector::kafka { + +using CppKafkaConsumerPtr = std::shared_ptr; + +/// Class for consume records from kafka topic +class KafkaConsumer { + public: + KafkaConsumer( + const CppKafkaConsumerPtr& consumer, + const uint32_t pollTimeOut, + const uint32_t pollBatchSize) + : consumer_(consumer), + pollTimeOutMillis_(pollTimeOut), + pollBatchSize_(pollBatchSize > 1 ? pollBatchSize : ConnectionConfig::defaultPollBatchSize) {} + + ~KafkaConsumer() {} + + /// Get the parititions of the given `topic`, and get the offset accroding to + /// the given `startupMode`. + const cppkafka::TopicPartitionList getTopicPartitions( + const std::string& topic, + const std::string& startupMode); + /// Set topic partitions offset according to the given `startupMode`. + const void setTopicPartitionsOffset( + cppkafka::TopicPartitionList& tps, + const std::string& startupMode); + /// Subscribe to the kafka topics. + void subscribe(const std::vector& topics); + /// Assign the consumer to the given topic partitions. + void assign(const cppkafka::TopicPartitionList& tps); + /// Consume a batch of messages. + const void consumeBatch(std::vector& msgs, size_t& msgBytes); + /// For test, Get the kafka topics that already subscribed. + const std::vector getSubscribedTopics(); + /// For test, Get the kafka assigned topic and partitions. + const cppkafka::TopicPartitionList getAssignedTopicPartitions(); + + private: + /// The kafka consume handle by using `cppkafka`. + CppKafkaConsumerPtr consumer_; + /// The timeout milliseconds for consuming kafka. + std::chrono::milliseconds pollTimeOutMillis_; + /// The batch size for consuming kafka. + uint32_t pollBatchSize_; +}; + +using KafkaConsumerPtr = std::shared_ptr; +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaDataSource.cpp b/velox/connectors/kafka/KafkaDataSource.cpp new file mode 100644 index 000000000000..70ee1beb03b6 --- /dev/null +++ b/velox/connectors/kafka/KafkaDataSource.cpp @@ -0,0 +1,165 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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 "velox/connectors/kafka/KafkaDataSource.h" +#include "velox/common/base/RuntimeMetrics.h" +#include "velox/connectors/kafka/KafkaTableHandle.h" +#include "velox/connectors/kafka/format/CSVRecordDeserializer.h" +#include "velox/connectors/kafka/format/RawRecordDeserializer.h" +#include "velox/connectors/kafka/format/StreamJSONRecordDeserializer.h" +#include "velox/connectors/kafka/KafkaConnectorSplit.h" +#include "velox/vector/BaseVector.h" + +namespace facebook::velox::connector::kafka { + +KafkaDataSource::KafkaDataSource( + const RowTypePtr& outputType, + const TableHandlePtr& tableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + const ConnectionConfigPtr& config) + : queryCtx_(connectorQueryCtx), + config_(config), + outputType_(outputType), + batchSize_(config_->getDataBatchSize()) { + VELOX_CHECK(batchSize_ > 0, "Batch size config value must greater than 0."); + const std::shared_ptr kafkaTableHandle = + std::dynamic_pointer_cast(tableHandle); + if (kafkaTableHandle) { + const std::unordered_map& tableParams = + kafkaTableHandle->tableParameters(); + config_ = config_->setConfigs(tableParams); + } else { + VELOX_FAIL( + "The table handle {} is not supported for kafka data source.", + tableHandle->connectorId()); + } + if (consumerCanbeCreated()) { + cppkafka::Configuration cppKafkaConfig = + config_->getCppKafkaConfiguration(); + createConsumer(cppKafkaConfig); + } + createCachedQueue(batchSize_); + createRecordDeserializer(config_->getFormat(), outputType_); +} + +bool KafkaDataSource::consumerCanbeCreated() { + return config_->exists(ConnectionConfig::kBootstrapServers) && + config_->exists(ConnectionConfig::kClientId) && + config_->exists(ConnectionConfig::kTopic) && + config_->exists(ConnectionConfig::kGroupId) && + config_->exists(ConnectionConfig::kFormat) && !consumer_.get(); +} + +void KafkaDataSource::createConsumer(cppkafka::Configuration& config) { + VELOX_CHECK_NULL( + consumer_.get(), + "Failed to create kafka consumer as the consumer is not null"); + CppKafkaConsumerPtr cppKafkaConsumer = + std::make_shared(config); + cppKafkaConsumer->set_destroy_flags(RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); + consumer_ = std::make_shared( + cppKafkaConsumer, config_->getPollTimeoutMills(), batchSize_); + std::string topic = config_->getTopic(); + topics_.emplace_back(topic); + consumer_->subscribe(topics_); +} + +void KafkaDataSource::createCachedQueue(const uint32_t size) { + VELOX_CHECK_GT( + size, 0, "Kafka consume message queue size must greater than 0"); + queue_.reserve(size); +} + +void KafkaDataSource::createRecordDeserializer( + const std::string& format, + const RowTypePtr& outputType) { + if (format == "json") { + deserializer_ = std::make_shared( + outputType, queryCtx_->memoryPool()); + } else if (format == "csv") { + deserializer_ = std::make_shared( + outputType, queryCtx_->memoryPool()); + } else if (format == "raw") { + deserializer_ = std::make_shared( + outputType, queryCtx_->memoryPool()); + } else { + VELOX_FAIL_UNSUPPORTED_INPUT_UNCATCHABLE( + "The data format {} is not supported for kafka.", format); + } + outRow_ = RowVector::createEmpty(outputType_, queryCtx_->memoryPool()); + outRow_->resize(1); +} + +void KafkaDataSource::addSplit(ConnectorSplitPtr split) { + KafkaConnectorSplit* kafkaConnectorSplit = + static_cast(split.get()); + VELOX_CHECK_NOT_NULL( + kafkaConnectorSplit, + "Failed to add split, because the kafka connector split is null."); + VELOX_CHECK_NOT_NULL( + consumer_.get(), + "Failed to add split, because the kafka consumer is null."); + cppkafka::TopicPartitionList topicPartitions = + kafkaConnectorSplit->getCppKafkaTopicPartitions(); + if (topicPartitions.size() == 0) { + const auto tps = + consumer_->getTopicPartitions(topics_[0], config_->getStartupMode()); + consumer_->assign(tps); + } else { + consumer_->setTopicPartitionsOffset(topicPartitions, config_->getStartupMode()); + consumer_->assign(topicPartitions); + } +} + +std::optional KafkaDataSource::next( + uint64_t, + velox::ContinueFuture&) { + std::optional res; + size_t consumedMsgBytes = 0; + if (queue_.empty()) { + // consume the data batch from kafka, and stored the consumed data in the queue. + consumer_->consumeBatch(queue_, consumedMsgBytes); + consumePos_ = 0; + // If nothing consumed, return directly. + if (consumedMsgBytes == 0) { + return res; + } + } + outRow_->prepareForReuse(); + // If batchSize > 1 and set `processDataSize = queue.size`, means to process the entrie batch that consumed and stored in the `queue` at once; + // If batchSize = 1 and set `processDataSize = 1`, means to process the consumed batch data one by one. + size_t processDataSize = batchSize_ > 1 ? queue_.size() : batchSize_; + outRow_->resize(processDataSize); + // Deserialize the consumed data. The `processDataSize` determines how many data would be deserialized at once. + for (size_t pos = 0; pos < processDataSize; ++pos) { + deserializer_->deserialize(queue_[pos + consumePos_], pos, outRow_); + completedBytes_ += queue_[pos + consumePos_].size(); + completedRows_ += 1; + } + res.emplace(std::dynamic_pointer_cast(outRow_)); + consumePos_ += processDataSize; + if (consumePos_ >= queue_.size()) { + queue_.clear(); + consumePos_ = 0; + } + return res; +} + +std::unordered_map KafkaDataSource::runtimeStats() { + std::unordered_map stats; + return stats; +} +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaDataSource.h b/velox/connectors/kafka/KafkaDataSource.h new file mode 100644 index 000000000000..02c6b7e76709 --- /dev/null +++ b/velox/connectors/kafka/KafkaDataSource.h @@ -0,0 +1,118 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "velox/common/base/RuntimeMetrics.h" +#include "velox/common/future/VeloxPromise.h" +#include "velox/connectors/Connector.h" +#include "velox/connectors/kafka/KafkaConfig.h" +#include "velox/connectors/kafka/KafkaConsumer.h" +#include "velox/type/Filter.h" +#include "velox/type/Type.h" +#include + +namespace facebook::velox::connector::kafka { + +using TableHandlePtr = std::shared_ptr; +using ConnectorSplitPtr = std::shared_ptr; + +class KafkaRecordDeserializer; +using KafkaRecordDeserializerPtr = std::shared_ptr; + +class KafkaDataSource : public DataSource { + public: + KafkaDataSource( + const RowTypePtr& outputType, + const TableHandlePtr& tableHandle, + const ConnectorQueryCtx* connectorQueryCtx, + const ConnectionConfigPtr& connectionConfig); + + /// Create a kafka connection to the given topics and partitions. + void addSplit(ConnectorSplitPtr split) override; + + /// Fetch record from the consumed records. + std::optional next(uint64_t size, velox::ContinueFuture& future) + override; + + void addDynamicFilter( + column_index_t outputChannel, + const std::shared_ptr& filter) override {} + + uint64_t getCompletedBytes() override { + return completedBytes_; + } + + uint64_t getCompletedRows() override { + return completedRows_; + } + + std::unordered_map runtimeStats() override; + + /// For test. + const KafkaConsumerPtr & getConsumer() const { + return consumer_; + } + + /// For test. + const KafkaRecordDeserializerPtr & getDeserializer() const { + return deserializer_; + } + + private: + /// The context of kafka connector for query. + const ConnectorQueryCtx* queryCtx_; + /// The connection config for kafka. + ConnectionConfigPtr config_; + /// The type of output. + RowTypePtr outputType_; + /// The kafka topics to be consumed. + std::vector topics_; + /// The kafka consumer. + KafkaConsumerPtr consumer_; + /// The kafka record deserializer. + KafkaRecordDeserializerPtr deserializer_; + /// Count how many rows consumed. + uint64_t completedRows_ = 0; + /// Count how many bytes consumed. + uint64_t completedBytes_ = 0; + /// The output row to be returned. + VectorPtr outRow_; + /// The batch size of data are consumed/processed at once . + /// If batchSize_ = 1, it means to process the consumed messages one by one, which likes valina flink does. + /// If batchSize_ > 1, it means to process the consumed messages by a entrie batch. + uint64_t batchSize_; + /// The cache queue for storing consumed data. + std::vector queue_; + /// The consumed position of the cache queue. + size_t consumePos_ = 0; + + /// Whether consumer can be created. + bool consumerCanbeCreated(); + + /// Create kafka consumer from the configuration. + void createConsumer(cppkafka::Configuration& config); + + /// Create message queue with given size. + void createCachedQueue(const uint32_t size); + + /// Create deserializer to deserialize the consumed recored to the given row + /// type. + void createRecordDeserializer( + const std::string& format, + const RowTypePtr& outputType); +}; + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaTableHandle.cpp b/velox/connectors/kafka/KafkaTableHandle.cpp new file mode 100644 index 000000000000..896a564dc132 --- /dev/null +++ b/velox/connectors/kafka/KafkaTableHandle.cpp @@ -0,0 +1,86 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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 "velox/connectors/kafka/KafkaTableHandle.h" + +namespace facebook::velox::connector::kafka { + +std::string KafkaTableHandle::toString() const { + std::stringstream out; + out << "table: " << tableName_; + if (dataColumns_) { + out << ", data columns: " << dataColumns_->toString(); + } + if (!tableParameters_.empty()) { + std::map orderedTableParameters{ + tableParameters_.begin(), tableParameters_.end()}; + out << ", table parameters: ["; + bool firstParam = true; + for (const auto& param : orderedTableParameters) { + if (!firstParam) { + out << ", "; + } + out << param.first << ":" << param.second; + firstParam = false; + } + out << "]"; + } + return out.str(); +} + +folly::dynamic KafkaTableHandle::serialize() const { + folly::dynamic obj = ConnectorTableHandle::serializeBase("KafkaTableHandle"); + obj["tableName"] = tableName_; + if (dataColumns_) { + obj["dataColumns"] = dataColumns_->serialize(); + } + folly::dynamic tableParameters = folly::dynamic::object; + for (const auto& param : tableParameters_) { + tableParameters[param.first] = param.second; + } + obj["tableParameters"] = tableParameters; + return obj; +} + +ConnectorTableHandlePtr KafkaTableHandle::create( + const folly::dynamic& obj, + void* context) { + auto connectorId = obj["connectorId"].asString(); + auto tableName = obj["tableName"].asString(); + RowTypePtr dataColumns; + if (auto it = obj.find("dataColumns"); it != obj.items().end()) { + dataColumns = ISerializable::deserialize(it->second, context); + } + + std::unordered_map tableParameters{}; + const auto& tableParametersObj = obj["tableParameters"]; + for (const auto& key : tableParametersObj.keys()) { + const auto& value = tableParametersObj[key]; + tableParameters.emplace(key.asString(), value.asString()); + } + + return std::make_shared( + connectorId, + tableName, + dataColumns, + tableParameters); +} + +void KafkaTableHandle::registerSerDe() { + auto& registry = DeserializationWithContextRegistryForSharedPtr(); + registry.Register("KafkaTableHandle", create); +} +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaTableHandle.h b/velox/connectors/kafka/KafkaTableHandle.h new file mode 100644 index 000000000000..9cab391987aa --- /dev/null +++ b/velox/connectors/kafka/KafkaTableHandle.h @@ -0,0 +1,76 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "velox/connectors/Connector.h" +#include "velox/core/ITypedExpr.h" +#include "velox/type/Filter.h" +#include "velox/type/Type.h" +#include + +namespace facebook::velox::connector::kafka { + +/// The table handle for kafka connector. +class KafkaTableHandle : public ConnectorTableHandle { + public: + KafkaTableHandle( + std::string connectorId, + std::string tableName, + const RowTypePtr& dataColumns = nullptr, + const std::unordered_map& tableParameters = {}) + : ConnectorTableHandle(connectorId), + tableName_(tableName), + dataColumns_(dataColumns), + tableParameters_(tableParameters) {} + + const std::string& tableName() { + return tableName_; + } + + const RowTypePtr& dataColumns() { + return dataColumns_; + } + + const std::unordered_map& tableParameters() { + return tableParameters_; + } + + std::unordered_map& getTableParameters() { + return tableParameters_; + } + + const std::string& name() const override { + return tableName_; + } + + std::string toString() const override; + + folly::dynamic serialize() const override; + + static ConnectorTableHandlePtr create( + const folly::dynamic& obj, + void* context); + + static void registerSerDe(); + + private: + std::string tableName_; + RowTypePtr dataColumns_; + std::unordered_map tableParameters_; +}; + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/CMakeLists.txt b/velox/connectors/kafka/format/CMakeLists.txt new file mode 100644 index 000000000000..12f9cc0a274c --- /dev/null +++ b/velox/connectors/kafka/format/CMakeLists.txt @@ -0,0 +1,21 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed 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. +velox_add_library(velox_kafka_connector_format OBJECT + JSONRecordDeserializer.cpp StreamJSONRecordDeserializer.cpp) + +velox_link_libraries( + velox_kafka_connector_format + velox_common_io + velox_connector + simdjson::simdjson) diff --git a/velox/connectors/kafka/format/CSVRecordDeserializer.h b/velox/connectors/kafka/format/CSVRecordDeserializer.h new file mode 100644 index 000000000000..ae1f389baa94 --- /dev/null +++ b/velox/connectors/kafka/format/CSVRecordDeserializer.h @@ -0,0 +1,37 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "velox/connectors/kafka/format/KafkaRecordDeserializer.h" + +namespace facebook::velox::connector::kafka { +/// Class for kafka record deserialization of csv format. +class KafkaCSVRecordDeserializer : public KafkaRecordDeserializer { + public: + KafkaCSVRecordDeserializer( + const RowTypePtr& outputType, + memory::MemoryPool* memoryPool) + : KafkaRecordDeserializer(outputType, memoryPool) {} + + const void deserialize( + const std::string & message, + const size_t index, + VectorPtr& vec) override { + VELOX_NYI("Not implemented."); + } +}; + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/JSONRecordDeserializer.cpp b/velox/connectors/kafka/format/JSONRecordDeserializer.cpp new file mode 100644 index 000000000000..2d82979a0bd0 --- /dev/null +++ b/velox/connectors/kafka/format/JSONRecordDeserializer.cpp @@ -0,0 +1,103 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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 "velox/connectors/kafka/format/JSONRecordDeserializer.h" + +namespace facebook::velox::connector::kafka { + +using IntDeserializer = BaseDeserializer; +using BigIntDeseralizer = BaseDeserializer; +using SmallIntDeserializer = BaseDeserializer; +using TinyIntDeserializer = BaseDeserializer; +using BoolDeserializer = BaseDeserializer; +using FloatDeserializer = BaseDeserializer; +using DoubleDeserializer = BaseDeserializer; +using StringDeserializer = BaseDeserializer; +using TimestampDeserializer = BaseDeserializer; + +const std::shared_ptr JSONDeserializer::create( + const TypePtr& type) { + const TypeKind kind = type->kind(); + if (kind == TypeKind::INTEGER) { + return std::make_shared(); + } else if (kind == TypeKind::BIGINT) { + return std::make_shared(); + } else if (kind == TypeKind::SMALLINT) { + return std::make_shared(); + } else if (kind == TypeKind::TINYINT) { + return std::make_shared(); + } else if (kind == TypeKind::BOOLEAN) { + return std::make_shared(); + } else if (kind == TypeKind::VARCHAR) { + return std::make_shared(); + } else if (kind == TypeKind::REAL) { + return std::make_shared(); + } else if (kind == TypeKind::DOUBLE) { + return std::make_shared(); + } else if (kind == TypeKind::TIMESTAMP) { + return std::make_shared(); + } else if (kind == TypeKind::ROW) { + const RowTypePtr rowType = std::dynamic_pointer_cast(type); + const std::vector fieldNames = rowType->names(); + const std::vector fieldTypes = rowType->children(); + std::vector> deserializers; + for (const auto& fieldType : fieldTypes) { + deserializers.emplace_back(create(fieldType)); + } + return std::make_shared( + fieldNames, fieldTypes, deserializers); + } else if (kind == TypeKind::ARRAY) { + const std::shared_ptr arrayType = + std::dynamic_pointer_cast(type); + const TypePtr& elementType = arrayType->elementType(); + const std::shared_ptr elementDeserializer = + create(elementType); + return std::make_shared( + elementType, elementDeserializer); + } else if (kind == TypeKind::MAP) { + const std::shared_ptr mapType = + std::dynamic_pointer_cast(type); + const TypePtr& keyType = mapType->keyType(); + const TypePtr& valueType = mapType->valueType(); + const std::shared_ptr keyDeserializer = create(keyType); + const std::shared_ptr valueDeserializer = + create(valueType); + return std::make_shared( + keyType, valueType, keyDeserializer, valueDeserializer); + } else { + VELOX_FAIL("The type is not supported: {}", type); + } +} + +const void KafkaJSONRecordDeserializer::deserialize( + const std::string& message, + const size_t index, + VectorPtr& vec) { + try { + Element element; + parser_->parse(message.data(), message.size()).get(element); + if (element.is_null()) { + vec->setNull(true, index); + } else { + deserializer_->deserialize(std::move(element), index, vec); + } + } catch (const std::exception& e) { + LOG(WARNING) << "Failed to deserialize record: " << message + << " , error: " << e.what(); + } +} + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/JSONRecordDeserializer.h b/velox/connectors/kafka/format/JSONRecordDeserializer.h new file mode 100644 index 000000000000..9f518d962e7c --- /dev/null +++ b/velox/connectors/kafka/format/JSONRecordDeserializer.h @@ -0,0 +1,160 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "velox/connectors/kafka/format/KafkaRecordDeserializer.h" +#include "velox/type/StringView.h" +#include "velox/type/Timestamp.h" +#include "velox/type/TimestampConversion.h" +#include "velox/type/Type.h" +#include "simdjson.h" +#include +#include + +namespace facebook::velox::connector::kafka { + +using Element = simdjson::dom::element; +using Elements = std::vector; + +struct JSONDeserializer { + public: + virtual inline const void + deserialize(const Element& e, const size_t index, VectorPtr& vec) = 0; + + static const std::shared_ptr create(const TypePtr& type); +}; + +template +struct BaseDeserializer : public JSONDeserializer { + public: + inline const void + deserialize(const Element& e, const size_t index, VectorPtr& vec) override { + auto flat = std::dynamic_pointer_cast>(vec); + if constexpr (std::is_same_v || std::is_same_v) { + flat->set(index, static_cast(e.get_double().value())); + } else if constexpr (std::is_same_v) { + flat->set(index, e.get_bool().value()); + } else if constexpr (std::is_same_v) { + const auto s = e.get_string().value(); + flat->set(index, StringView(s.data(), s.size())); + } else if constexpr (std::is_same_v) { + const auto s = e.get_string().value(); + const auto timestamp = + util::fromTimestampString( + s.data(), s.size(), util::TimestampParseMode::kLegacyCast) + .thenOrThrow(folly::identity, [&](const Status& status) { + VELOX_FAIL("error while parse timestamp: {}", status.message()); + }); + flat->set(index, timestamp); + } else if constexpr ( + std::is_same_v || std::is_same_v || + std::is_same_v || std::is_same_v) { + flat->set(index, static_cast(e.get_int64().value())); + } else { + VELOX_FAIL("The type {} is not supported", typeid(T).name()); + } + } +}; + +struct RowDeserializer : public JSONDeserializer { + public: + RowDeserializer( + const std::vector& fieldNames, + const std::vector& fieldTypes, + const std::vector>& deserializers) + : fieldNames_(fieldNames), + fieldTypes_(fieldTypes), + deserializers_(deserializers) {} + + inline const void + deserialize(const Element& e, const size_t index, VectorPtr& vec) override { + RowVectorPtr rowVector = std::dynamic_pointer_cast(vec); + std::vector& rowFields = rowVector->children(); + for (size_t i = 0; i < fieldNames_.size(); ++i) { + if (e[fieldNames_[i]].is_null()) { + rowFields[i]->setNull(index, true); + } else { + deserializers_[i]->deserialize(e[fieldNames_[i]], index, rowFields[i]); + } + } + } + + private: + std::vector fieldNames_; + std::vector fieldTypes_; + std::vector> deserializers_; +}; + +struct ArrayDeserializer : public JSONDeserializer { + public: + ArrayDeserializer( + const TypePtr& elementType, + const std::shared_ptr& deserializer) + : elementType_(elementType), elementDeserializer_(deserializer) {} + + inline const void + deserialize(const Element& e, const size_t index, VectorPtr& vec) override {} + + private: + TypePtr elementType_; + std::shared_ptr elementDeserializer_; +}; + +struct MapDeserializer : public JSONDeserializer { + public: + MapDeserializer( + const TypePtr& keyType, + const TypePtr& valueType, + const std::shared_ptr& keyDeserializer, + const std::shared_ptr& valueDeserializer) + : keyType_(keyType), + valueType_(valueType), + keyDeserializer_(keyDeserializer), + valueDeserializer_(valueDeserializer) {} + + inline const void + deserialize(const Element& e, const size_t index, VectorPtr& vec) override {} + + private: + TypePtr keyType_; + TypePtr valueType_; + std::shared_ptr keyDeserializer_; + std::shared_ptr valueDeserializer_; +}; + +/// Class for kafka record deserialization of json format. +class KafkaJSONRecordDeserializer : public KafkaRecordDeserializer { + public: + KafkaJSONRecordDeserializer( + const RowTypePtr& outputType, + memory::MemoryPool* memoryPool) + : KafkaRecordDeserializer(outputType, memoryPool), + deserializer_(std::dynamic_pointer_cast( + JSONDeserializer::create(outputType))), + parser_(std::make_shared()) {} + + const void deserialize( + const std::string& message, + const size_t index, + VectorPtr& vec) override; + + private: + std::shared_ptr deserializer_; + std::shared_ptr parser_; +}; + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/KafkaRecordDeserializer.h b/velox/connectors/kafka/format/KafkaRecordDeserializer.h new file mode 100644 index 000000000000..b86b91dace61 --- /dev/null +++ b/velox/connectors/kafka/format/KafkaRecordDeserializer.h @@ -0,0 +1,55 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "velox/common/memory/MemoryPool.h" +#include "velox/type/Type.h" +#include "velox/vector/ComplexVector.h" +#include "velox/vector/FlatVector.h" + +namespace facebook::velox::connector::kafka { + +/// Base class for kafka record deserialization. +class KafkaRecordDeserializer { + public: + KafkaRecordDeserializer( + const RowTypePtr& outputType, + memory::MemoryPool* memoryPool) + : outputType_(outputType), memoryPool_(memoryPool) { + VELOX_CHECK_GT( + outputType_->size(), + 0, + "Output type size of record deserializer must great than 0."); + VELOX_CHECK_NOT_NULL( + memoryPool_, "Memory pool of record deserializer must not be null."); + } + + virtual ~KafkaRecordDeserializer() = default; + + /// Deserialize a single message. + virtual const void deserialize( + const std::string & message, + const size_t index, + VectorPtr& vec) = 0; + + protected: + RowTypePtr outputType_; + memory::MemoryPool* memoryPool_; +}; + +using KafkaRecordDeserializerPtr = std::shared_ptr; + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/RawRecordDeserializer.h b/velox/connectors/kafka/format/RawRecordDeserializer.h new file mode 100644 index 000000000000..57371b2827ea --- /dev/null +++ b/velox/connectors/kafka/format/RawRecordDeserializer.h @@ -0,0 +1,56 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "velox/connectors/kafka/format/KafkaRecordDeserializer.h" +#include "velox/type/Type.h" + +namespace facebook::velox::connector::kafka { +/// Class for kafka record deserialization of raw format. +class KafkaRawRecordDeserializer : public KafkaRecordDeserializer { + public: + KafkaRawRecordDeserializer( + const RowTypePtr& outputType, + memory::MemoryPool* memoryPool) + : KafkaRecordDeserializer(outputType, memoryPool) { + VELOX_CHECK_EQ( + outputType_->size(), + 1, + "Output type size of raw deserializer must be 1."); + const TypePtr& childType = outputType_->childAt(0); + VELOX_CHECK_EQ( + childType->kind(), + TypeKind::VARCHAR, + "Output type must be Row(String)."); + } + + const void deserialize( + const std::string& message, + const size_t index, + VectorPtr& vec) override { + RowVectorPtr rowVector = std::dynamic_pointer_cast(vec); + VELOX_CHECK_EQ( + rowVector->childrenSize(), + 1, + "The raw record vector children size {} is not 1", + rowVector->childrenSize()); + VectorPtr& childVector = rowVector->children()[0]; + auto flat = std::dynamic_pointer_cast>(childVector); + flat->set(index, StringView(message.data(), message.size())); + } +}; + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/StreamJSONRecordDeserializer.cpp b/velox/connectors/kafka/format/StreamJSONRecordDeserializer.cpp new file mode 100644 index 000000000000..fdb29a180e22 --- /dev/null +++ b/velox/connectors/kafka/format/StreamJSONRecordDeserializer.cpp @@ -0,0 +1,71 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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 "velox/connectors/kafka/format/StreamJSONRecordDeserializer.h" + +namespace facebook::velox::connector::kafka { + +const std::shared_ptr StreamJSONDeserializer::create( + const TypePtr& type) { + const TypeKind kind = type->kind(); + if (kind == TypeKind::INTEGER) { + return std::make_shared(); + } else if (kind == TypeKind::BIGINT) { + return std::make_shared(); + } else if (kind == TypeKind::HUGEINT) { + return std::make_shared(); + } else if (kind == TypeKind::VARCHAR) { + return std::make_shared(); + } else if (kind == TypeKind::TIMESTAMP) { + return std::make_shared(); + } else if (kind == TypeKind::ROW) { + const RowTypePtr rowType = std::dynamic_pointer_cast(type); + const std::vector fieldNames = rowType->names(); + const std::vector fieldTypes = rowType->children(); + std::vector> deserializers; + for (const auto& fieldType : fieldTypes) { + deserializers.emplace_back(create(fieldType)); + } + return std::make_shared( + fieldNames, fieldTypes, deserializers); + } else { + VELOX_FAIL("The type is not supported: {}", type); + } +} + +const void KafkaStreamJSONRecordDeserializer::deserialize( + const std::string& message, + const size_t index, + VectorPtr& vec) { + try { + simdjson::padded_string_view json_padded( + message.data(), + message.size(), + message.size() + simdjson::SIMDJSON_PADDING); + simdjson::ondemand::document doc = parser_->iterate(json_padded); + JSONValue value = doc.get_value(); + if (value.is_null()) { + vec->setNull(index, true); + } else { + deserializer_->deserialize(value, index, vec); + } + } catch (const std::exception& e) { + LOG(WARNING) << "Failed to deserialize record: " << message + << " , error: " << e.what(); + } +} + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/StreamJSONRecordDeserializer.h b/velox/connectors/kafka/format/StreamJSONRecordDeserializer.h new file mode 100644 index 000000000000..0f1b7971868c --- /dev/null +++ b/velox/connectors/kafka/format/StreamJSONRecordDeserializer.h @@ -0,0 +1,165 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include "velox/connectors/kafka/format/StreamJSONRecordDeserializer.h" +#include "velox/connectors/kafka/format/KafkaRecordDeserializer.h" + +#include "velox/type/StringView.h" +#include "velox/type/Timestamp.h" +#include "velox/type/Type.h" +#include "simdjson.h" +#include +#include + +namespace facebook::velox::connector::kafka { + +using JSONDoc = simdjson::ondemand::document; +using JSONValue = simdjson::ondemand::value; +using JSONRow = simdjson::ondemand::object; +using JSONArray = simdjson::ondemand::array; + +struct StreamJSONDeserializer { + public: + virtual ~StreamJSONDeserializer() = default; + + static const std::shared_ptr create( + const TypePtr& type); + + virtual const void + deserialize(JSONValue& e, const size_t index, VectorPtr& vec) = 0; +}; + +struct StreamIntDeserializer : public StreamJSONDeserializer { + public: + StreamIntDeserializer() {} + + inline const void + deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { + auto flat = std::dynamic_pointer_cast>(vec); + flat->set(index, static_cast(e.get_int64())); + } +}; + +struct StreamBigIntDeserializer : public StreamJSONDeserializer { + public: + StreamBigIntDeserializer() {} + + inline const void + deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { + auto flat = std::dynamic_pointer_cast>(vec); + flat->set(index, e.get_int64()); + } +}; + +struct StreamHugeIntDeserializer : public StreamJSONDeserializer { + public: + StreamHugeIntDeserializer() {} + + inline const void + deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { + auto flat = std::dynamic_pointer_cast>(vec); + flat->set(index, static_cast(e.get_int64())); + } +}; + +struct StreamStringDeserializer : public StreamJSONDeserializer { + public: + StreamStringDeserializer() {} + + inline const void + deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { + auto flat = + std::dynamic_pointer_cast>(vec); + std::string_view s = e.get_string(); + flat->set(index, facebook::velox::StringView(s.data(), s.size())); + } +}; + +struct StreamTimestampDeserializer : public StreamJSONDeserializer { + public: + StreamTimestampDeserializer() {} + + inline const void + deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { + auto flat = + std::dynamic_pointer_cast>(vec); + std::string_view s = e.get_string(); + const auto timestamp = + util::fromTimestampString( + s.data(), s.size(), util::TimestampParseMode::kLegacyCast) + .thenOrThrow(folly::identity, [&](const Status& status) { + VELOX_FAIL("error while parse timestamp: {}", status.message()); + }); + flat->set(index, timestamp); + } +}; + +struct StreamRowDeserializer : public StreamJSONDeserializer { + public: + StreamRowDeserializer( + const std::vector& fieldNames, + const std::vector& fieldTypes, + const std::vector>& deserializers) + : fieldNames_(fieldNames), + fieldTypes_(fieldTypes), + deserializers_(deserializers) {} + + inline const void + deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { + RowVectorPtr rowVector = std::dynamic_pointer_cast(vec); + std::vector& rowFields = rowVector->children(); + JSONRow row = e.get_object(); + for (size_t i = 0; i < fieldNames_.size(); ++i) { + JSONValue v; + auto err = row[fieldNames_[i]].get(v); + if (err != simdjson::error_code::SUCCESS || v.is_null()) { + rowFields[i]->setNull(index, true); + } else { + deserializers_[i]->deserialize(v, index, rowFields[i]); + } + } + } + + private: + std::vector fieldNames_; + std::vector fieldTypes_; + std::vector> deserializers_; +}; + +/// Class for kafka record deserialization of json format, using the streaming +/// interface of simdjson +class KafkaStreamJSONRecordDeserializer : public KafkaRecordDeserializer { + public: + KafkaStreamJSONRecordDeserializer( + const RowTypePtr& outputType, + memory::MemoryPool* memoryPool) + : KafkaRecordDeserializer(outputType, memoryPool), + deserializer_(std::dynamic_pointer_cast( + StreamJSONDeserializer::create(outputType))), + parser_(std::make_shared()) {} + + const void deserialize( + const std::string& message, + const size_t index, + VectorPtr& vec) override; + + private: + std::shared_ptr deserializer_; + std::shared_ptr parser_; +}; + +} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/tests/CMakeLists.txt b/velox/connectors/kafka/tests/CMakeLists.txt new file mode 100644 index 000000000000..6a21de039f9e --- /dev/null +++ b/velox/connectors/kafka/tests/CMakeLists.txt @@ -0,0 +1,27 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed 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. +add_executable(velox_kafka_connector_test KafkaConnectorTest.cpp) + +add_test(velox_kafka_connector_test velox_kafka_connector_test) + +target_link_libraries( + velox_kafka_connector_test + velox_kafka_connector + velox_kafka_connector_format + velox_vector_test_lib + velox_exec_test_lib + velox_exec + velox_common_base + GTest::gtest + GTest::gtest_main) diff --git a/velox/connectors/kafka/tests/KafkaConnectorTest.cpp b/velox/connectors/kafka/tests/KafkaConnectorTest.cpp new file mode 100644 index 000000000000..7bb01897d631 --- /dev/null +++ b/velox/connectors/kafka/tests/KafkaConnectorTest.cpp @@ -0,0 +1,163 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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 "velox/connectors/Connector.h" +#include "velox/connectors/kafka/KafkaDataSource.h" +#include "velox/connectors/kafka/format/StreamJSONRecordDeserializer.h" +#include "velox/connectors/kafka/tests/KafkaConnectorTestBase.h" +#include "velox/vector/ComplexVector.h" +#include "velox/type/Timestamp.h" +#include "velox/type/StringView.h" +#include +#include + +namespace facebook::velox::connector::kafka::test { + +class KafkaConnectorTest : public KafkaConnectorTestBase {}; + +TEST_F(KafkaConnectorTest, testConfig) { + std::shared_ptr kafkaConnector = getConnector(kKafkaConnectorId); + ASSERT_TRUE(kafkaConnector != nullptr); + const std::shared_ptr& connectorConfig = kafkaConnector->connectorConfig(); + const std::shared_ptr kafkaConfig = std::make_shared(connectorConfig); + ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kBootstrapServers)); + ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kTopic)); + ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kClientId)); + ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kGroupId)); + ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kFormat)); +} + +TEST_F(KafkaConnectorTest, testKafkaConsumer) { + const std::unique_ptr dataSource = createDataSource(); + KafkaDataSource * kafkaDataSource = reinterpret_cast(dataSource.get()); + ASSERT_TRUE(kafkaDataSource != nullptr); + const auto & kafkaConsumer = kafkaDataSource->getConsumer(); + ASSERT_TRUE(kafkaConsumer != nullptr); + std::vector topics = kafkaConsumer->getSubscribedTopics(); + ASSERT_TRUE(topics.size() > 0); + ASSERT_TRUE(topics[0] == kafkaTopic); +} + +TEST_F(KafkaConnectorTest, testAssignPartitions) { + const std::unique_ptr dataSource = createDataSource(); + KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); + ASSERT_TRUE(kafkaDataSource != nullptr); + kafkaDataSource->addSplit(createKafkaSplit()); + const auto & kafkaConsumer = kafkaDataSource->getConsumer(); + ASSERT_TRUE(kafkaConsumer != nullptr); + const cppkafka::TopicPartitionList tps = kafkaConsumer->getAssignedTopicPartitions(); + ASSERT_TRUE(tps.size() > 0); + const cppkafka::TopicPartition tp = tps[0]; + ASSERT_TRUE(tp.get_topic() == kafkaTopic); + ASSERT_TRUE(tp.get_partition() == 0); + ASSERT_TRUE(tp.get_offset() > 0); +} + +TEST_F(KafkaConnectorTest, testConsumeMessages) { + const std::unique_ptr dataSource = createDataSource(); + KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); + ASSERT_TRUE(kafkaDataSource != nullptr); + kafkaDataSource->addSplit(createKafkaSplit()); + std::string testMsg = "This is a test message!"; + sendMessageToKafka(testMsg); + const auto & kafkaConsumer = kafkaDataSource->getConsumer(); + ASSERT_TRUE(kafkaConsumer != nullptr); + std::vector msgs; + size_t msgBytes = 0; + kafkaConsumer->consumeBatch(msgs, msgBytes); + ASSERT_TRUE(msgs.size() == 1); + ASSERT_TRUE(msgs[0] == testMsg); + ASSERT_TRUE(msgBytes == testMsg.size()); +} + +TEST_F(KafkaConnectorTest, testCreateDeserializer) { + const std::unique_ptr dataSource = createDataSource(); + KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); + ASSERT_TRUE(kafkaDataSource != nullptr); + const auto & deserializer = kafkaDataSource->getDeserializer(); + const std::shared_ptr jsonDeserializer = + std::dynamic_pointer_cast(deserializer); + ASSERT_TRUE(jsonDeserializer != nullptr); +} + +TEST_F(KafkaConnectorTest, testDeserializeMessages) { + const std::unique_ptr dataSource = createDataSource(); + KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); + ASSERT_TRUE(kafkaDataSource != nullptr); + const auto & deserializer = kafkaDataSource->getDeserializer(); + std::string msg = "{\"event_type\":1, \"bid\": {\"auction\":1, \"bidder\":222, \"price\":1113, \"channel\":\"OTS\", \"url\":\"http://testkafka/a/b/c\", \"dateTime\":\"2025-06-18 11:22:33\", \"extra\":\"xxxx\"}}"; + VectorPtr vec1 = RowVector::createEmpty(outputType, memoryPool.get()); + vec1->resize(1); + deserializer->deserialize(msg, 0, vec1); + std::shared_ptr rowVector1 = std::dynamic_pointer_cast(vec1); + ASSERT_TRUE(rowVector1 != nullptr); + ASSERT_TRUE(rowVector1->size() == 1); + ASSERT_TRUE(rowVector1->children().size() == 2); + std::shared_ptr> flat = std::dynamic_pointer_cast>(rowVector1->childAt(0)); + ASSERT_TRUE(flat != nullptr); + ASSERT_TRUE(flat->size() == 1); + ASSERT_TRUE(flat->valueAt(0) == 1); + std::shared_ptr subRow = std::dynamic_pointer_cast(rowVector1->childAt(1)); + ASSERT_TRUE(subRow != nullptr); + ASSERT_TRUE(subRow->size() == 1); + ASSERT_TRUE(subRow->children().size() == 7); + std::shared_ptr> f1 = std::dynamic_pointer_cast>(subRow->childAt(0)); + std::shared_ptr> f2 = + std::dynamic_pointer_cast>(subRow->childAt(3)); + std::shared_ptr> f3 = + std::dynamic_pointer_cast>(subRow->childAt(5)); + ASSERT_TRUE(f1 != nullptr && f2 != nullptr && f3 != nullptr); + ASSERT_TRUE(f1->valueAt(0) == 1); + ASSERT_TRUE(f2->valueAt(0).str() == "OTS"); + ASSERT_TRUE(f3->valueAt(0).toMillis() == 1750245753000); +} + +TEST_F(KafkaConnectorTest, testKafkaSourceNext) { + const std::unique_ptr dataSource = createDataSource(); + KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); + ASSERT_TRUE(kafkaDataSource != nullptr); + kafkaDataSource->addSplit(createKafkaSplit()); + std::string msg1 = "{\"event_type\":1, \"bid\": {\"auction\":1, \"bidder\":222, \"price\":1113, \"channel\":\"OTS\", \"url\":\"http://testkafka/a/b/c\", \"dateTime\":\"2025-06-18 11:22:33\", \"extra\":\"xxxx\"}}"; + std::string msg2 = "{\"event_type\":2, \"bid\": {\"auction\":2, \"bidder\":223, \"price\":1114, \"channel\":\"OTS-1\", \"url\":\"http://testkafka/a/b/c\", \"dateTime\":\"2025-06-18 11:22:33\", \"extra\":\"xxxx\"}}"; + std::string msg3 = "{\"event_type\":3, \"bid\": {\"auction\":3, \"bidder\":225, \"price\":1115, \"channel\":\"OTS-2\", \"url\":\"http://testkafka/a/b/c\", \"dateTime\":\"2025-06-18 11:22:33\", \"extra\":\"xxxx\"}}"; + sendMessageToKafka(msg1); + sendMessageToKafka(msg2); + sendMessageToKafka(msg3); + auto future = facebook::velox::ContinueFuture{folly::Unit{}}; + std::optional res = kafkaDataSource->next(0, future); + ASSERT_TRUE(res.value() != nullptr); + RowVectorPtr rowVector = res.value(); + ASSERT_TRUE(rowVector->size() == 3); + std::shared_ptr> flat = std::dynamic_pointer_cast>(rowVector->childAt(0)); + std::shared_ptr subRow = std::dynamic_pointer_cast(rowVector->childAt(1)); + ASSERT_TRUE(flat != nullptr && flat->size() == 3); + ASSERT_TRUE(subRow != nullptr && subRow->size() == 3); + std::shared_ptr> subF1 = std::dynamic_pointer_cast>(subRow->childAt(0)); + ASSERT_TRUE(subF1 != nullptr && subF1->size() == 3); + for (size_t i = 0; i < 3; ++i) { + ASSERT_TRUE(flat->valueAt(i) == i+1); + ASSERT_TRUE(subF1->valueAt(i) == i+1); + } +} + +} // namespace facebook::velox::connector::kafka::test + +int main(int argc, char* argv[]) { + testing::InitGoogleTest(&argc, argv); + folly::Init init(&argc, &argv, false); + gflags::ParseCommandLineFlags(&argc, &argv, true); // Parse gflags + return RUN_ALL_TESTS(); +} diff --git a/velox/connectors/kafka/tests/KafkaConnectorTestBase.h b/velox/connectors/kafka/tests/KafkaConnectorTestBase.h new file mode 100644 index 000000000000..3cd66a0946db --- /dev/null +++ b/velox/connectors/kafka/tests/KafkaConnectorTestBase.h @@ -0,0 +1,184 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include "velox/connectors/kafka/KafkaConnector.h" +#include "velox/connectors/kafka/KafkaConnectorSplit.h" +#include "velox/connectors/kafka/KafkaTableHandle.h" +#include "velox/connectors/kafka/KafkaConfig.h" +#include "velox/type/Type.h" +#include "velox/type/Filter.h" +#include "velox/exec/tests/utils/OperatorTestBase.h" +#include +#include + +namespace facebook::velox::connector::kafka::test { + +class KafkaConnectorTestBase : public exec::test::OperatorTestBase { + public: + std::string kafkaInstance = "localhost:9092"; + std::string kafkaTopic = "test_kafka"; + const std::string kafkaConsumeGroupId = "test_kafka_group_id"; + const std::string kafkaDataFormat = "json"; + const std::string kafkaClientId = "test_kafka_client"; + const std::string kafkaAutoOffsetReset = "earliest"; + const std::string kafkaStartupMode = "latest-offsets"; + const std::string kKafkaConnectorId = "test-kafka"; + const std::shared_ptr outputType = createOutputType(); + const std::shared_ptr memoryPool = memory::memoryManager()->addLeafPool(); + + void SetUp() override { + OperatorTestBase::SetUp(); + init(); + connector::registerConnectorFactory( + std::make_shared()); + std::unordered_map configMap; + configMap[connector::kafka::ConnectionConfig::kBootstrapServers] = kafkaInstance; + configMap[connector::kafka::ConnectionConfig::kGroupId] = kafkaConsumeGroupId; + configMap[connector::kafka::ConnectionConfig::kFormat] = kafkaDataFormat; + configMap[connector::kafka::ConnectionConfig::kClientId] = kafkaClientId; + configMap[connector::kafka::ConnectionConfig::kTopic] = kafkaTopic; + configMap[connector::kafka::ConnectionConfig::kAutoResetOffset] = kafkaAutoOffsetReset; + configMap[connector::kafka::ConnectionConfig::kStartupMode] = kafkaStartupMode; + std::shared_ptr config = + std::make_shared(std::move(configMap)); + auto kafkaConnector = + connector::getConnectorFactory( + connector::kafka::KafkaConnectorFactory::kKafkaConnectorName) + ->newConnector(kKafkaConnectorId, config); + connector::registerConnector(kafkaConnector); + } + + void TearDown() override { + connector::unregisterConnector(kKafkaConnectorId); + connector::unregisterConnectorFactory( + connector::kafka::KafkaConnectorFactory::kKafkaConnectorName); + OperatorTestBase::TearDown(); + } + + void init() { + char* currentPath; + if ((currentPath = getcwd(NULL, 0)) == NULL) { + VELOX_FAIL("Failed to get curent path"); + } + std::string kafkaConfName = "kafka.conf"; + char kafkaConfPath[strlen(currentPath) + kafkaConfName.size() + 1]; + sprintf(kafkaConfPath, "%s%s%s", currentPath, "/", kafkaConfName.data()); + std::ifstream kafkaConfFile(kafkaConfPath); + if (!kafkaConfFile.is_open()) { + VELOX_FAIL("Failed to open kafka config file: {}", std::string(kafkaConfPath, strlen(kafkaConfPath))); + } + std::string kKafkaTestInstance = "kafka.test.instance"; + std::string kKafkaTestTopic = "kafka.test.topic"; + std::string confLine; + while(getline(kafkaConfFile, confLine)) { + if (confLine.empty()) { + continue; + } + if (confLine.find(kKafkaTestInstance) != std::string::npos) { + kafkaInstance = confLine.substr(kKafkaTestInstance.size() + 1); + } else if (confLine.find(kKafkaTestTopic) != std::string::npos) { + kafkaTopic = confLine.substr(kKafkaTestTopic.size() + 1); + } + } + kafkaConfFile.close(); + } + + /// Row>> + static const std::shared_ptr createOutputType() { + std::vector bidRowFieldNames = {"auction", "bidder", "price", "channel", "url", "dateTime", "extra"}; + std::vector bidRowFieldTypes = { + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared(), + std::make_shared() + }; + std::shared_ptr bidRowType = std::make_shared(std::move(bidRowFieldNames), std::move(bidRowFieldTypes)); + std::vector outputFieldNames = {"event_type", "bid"}; + std::vector outputFieldTypes = { + std::make_shared(), + bidRowType + }; + return std::make_shared(std::move(outputFieldNames), std::move(outputFieldTypes)); + } + + const std::shared_ptr createKafkaSplit() { + std::unordered_map>> topicPartitions; + std::vector> partitionOffsets { std::pair(0, 0) }; + topicPartitions[kafkaTopic] = partitionOffsets; + return std::make_shared( + kKafkaConnectorId, + kafkaInstance, + kafkaConsumeGroupId, + kafkaDataFormat, + false, + "earliest", + topicPartitions + ); + } + + const std::shared_ptr createKafkaTableHandle() { + return std::make_shared( + kKafkaConnectorId, + kafkaTopic, + outputType); + } + + const std::shared_ptr createQueryCtx() { + const auto kafkaConnector = getConnector(kKafkaConnectorId); + const auto connectorConfig = kafkaConnector->connectorConfig(); + std::shared_ptr connectorQueryCtx = + std::make_shared( + memoryPool.get(), + nullptr, + connectorConfig.get(), + nullptr, + common::PrefixSortConfig(), + nullptr, + nullptr, + "query.Kafka", + "task.Kafka", + "planNodeId.Kafka", + 0, + ""); + return connectorQueryCtx; + } + + const std::unique_ptr createDataSource() { + std::shared_ptr kafkaConnector = getConnector(kKafkaConnectorId); + const std::shared_ptr kafkaTableHandle = createKafkaTableHandle(); + std::unordered_map> columnHandles; + return kafkaConnector->createDataSource(outputType, kafkaTableHandle, columnHandles, createQueryCtx().get()); + } + + const void sendMessageToKafka(const std::string & message) { + cppkafka::Configuration config = {{"metadata.broker.list", kafkaInstance}}; + cppkafka::Producer producer(config); + cppkafka::MessageBuilder builder(kafkaTopic); + builder.partition(0); + builder.payload(message); + producer.produce(builder); + producer.flush(); + sleep(1); + } +}; + +} // namespace facebook::velox::connector::kafka::test diff --git a/velox/connectors/kafka/tests/kafka.conf b/velox/connectors/kafka/tests/kafka.conf new file mode 100644 index 000000000000..748f310e103d --- /dev/null +++ b/velox/connectors/kafka/tests/kafka.conf @@ -0,0 +1,2 @@ +kafka.test.instance=localhost:9092 +kafka.test.topic=test_kafka diff --git a/velox/experimental/stateful/GroupWindowAggregator.cpp b/velox/experimental/stateful/GroupWindowAggregator.cpp index 2e03df5b99c7..62bc47615342 100644 --- a/velox/experimental/stateful/GroupWindowAggregator.cpp +++ b/velox/experimental/stateful/GroupWindowAggregator.cpp @@ -82,12 +82,12 @@ void GroupWindowAggregator::getOutput() { } // 1. Partition input by key - std::map keyToData = keySelector_->partition(input_); + std::map keyToData = keySelector_->partition(input_); for (const auto& [key, keyedData] : keyToData) { // 2. Set the current key in the context windowContext_->setCurrentKey(key); // 3. Partition the keyed data by rowtime or processing time - std::map timestampToData = sliceAssigner_->assignSliceEnd(keyedData); + std::map timestampToData = sliceAssigner_->assignSliceEnd(keyedData); for (const auto& [timestamp, data] : timestampToData) { // 4. Assign data to window std::vector windows = diff --git a/velox/experimental/stateful/InternalPriorityQueue.h b/velox/experimental/stateful/InternalPriorityQueue.h index 44a1e7bbb95e..41a21042999b 100644 --- a/velox/experimental/stateful/InternalPriorityQueue.h +++ b/velox/experimental/stateful/InternalPriorityQueue.h @@ -31,6 +31,8 @@ class InternalPriorityQueue { virtual T peek() = 0; virtual void clear() = 0; + + virtual bool empty() = 0; }; // This class is relevent to flink HeapPriorityQueue. @@ -73,6 +75,10 @@ class HeapPriorityQueue : public InternalPriorityQueue { size_ = 0; } + bool empty() override { + return size_ == 0; + } + void remove(T toRemove) { // Implementation for removing an element from the priority queue set auto it = std::find(queue_.begin(), queue_.end(), toRemove); diff --git a/velox/experimental/stateful/InternalTimerService.h b/velox/experimental/stateful/InternalTimerService.h index 10fc11f27216..39c3d102737a 100644 --- a/velox/experimental/stateful/InternalTimerService.h +++ b/velox/experimental/stateful/InternalTimerService.h @@ -15,12 +15,15 @@ */ #pragma once +#include #include #include #include #include #include #include +#include +#include namespace facebook::velox::stateful { @@ -43,13 +46,14 @@ class InternalTimerService { std::shared_ptr> oldHead = processingTimeTimersQueue_.peek(); processingTimeTimersQueue_.add(std::make_shared>(time, key, ns)); long nextTriggerTime = oldHead != nullptr ? oldHead->timestamp() : std::numeric_limits::max() ; + std::cout << "time:" << time << " nextTriggerTime:" << nextTriggerTime << std::endl; if (time < nextTriggerTime) { - if (nextTimer_.hasValue() && !nextTimer_.isReady()) { - nextTimer_.cancel(); + if (nextTimer_.has_value()) { + processingTimeService_->cancel(nextTimer_.value()); } - nextTimer_ = processingTimeService_->registerTimer(time, [&](long time) { - onProcessingTime(time); - }); + nextTimer_ = processingTimeService_->registerTimer(time, ProcessingTimerTask(time, [&](long processingTime) { + onProcessingTime(processingTime); + })); } } @@ -80,27 +84,43 @@ class InternalTimerService { void close() { eventTimeTimersQueue_.clear(); + processingTimeTimersQueue_.clear(); + processingTimeService_->close(); } private: void onProcessingTime(long time) { - nextTimer_ = folly::Future::makeEmpty(); + std::string taskName = ""; + if (nextTimer_.has_value()) { + taskName = nextTimer_.value(); + } + nextTimer_ = std::nullopt; std::shared_ptr> timer = nullptr; - while ((timer = processingTimeTimersQueue_.peek()) != nullptr - && timer->timestamp() <= time) { + bool triggerOnProcessingTime = true; + while (triggerOnProcessingTime && !processingTimeTimersQueue_.empty()) { + timer = processingTimeTimersQueue_.peek(); + if (!timer || timer->timestamp() > time) { + triggerOnProcessingTime = false; + continue; + } processingTimeTimersQueue_.poll(); triggerable_->onProcessingTime(timer); + timer = nullptr; + } + + if (!taskName.empty()) { + processingTimeService_->finish(taskName); } - if (timer != nullptr && !nextTimer_.hasValue()) { - nextTimer_ = processingTimeService_->registerTimer(timer->timestamp(), [&](long time) { - onProcessingTime(time); - }); + if (timer != nullptr && !nextTimer_.has_value()) { + nextTimer_ = processingTimeService_->registerTimer(timer->timestamp(), ProcessingTimerTask(timer->timestamp(), [&](long processingTime) { + onProcessingTime(processingTime); + })); } } Triggerable* triggerable_; - folly::Future nextTimer_; + std::optional nextTimer_; std::shared_ptr processingTimeService_; HeapPriorityQueue>> eventTimeTimersQueue_; HeapPriorityQueue>> processingTimeTimersQueue_; diff --git a/velox/experimental/stateful/KeySelector.cpp b/velox/experimental/stateful/KeySelector.cpp index 9f81f30cd8db..624dde9a40ae 100644 --- a/velox/experimental/stateful/KeySelector.cpp +++ b/velox/experimental/stateful/KeySelector.cpp @@ -26,9 +26,9 @@ KeySelector::KeySelector( numPartitions_(numPartitions) { } -std::map KeySelector::partition(const RowVectorPtr& input) { +std::map KeySelector::partition(const RowVectorPtr& input) { if (numPartitions_ == 1) { - return std::map{{0, input}}; + return std::map{{0, input}}; } prepareForInput(input); @@ -39,7 +39,7 @@ std::map KeySelector::partition(const RowVectorPtr& inpu // TODO: this is a optimization, as the RowVector may have be partitioned in // local aggregation, so need not to partition again in global agg, but need // to verify whether the judge condition is enough. - return std::map{{*part, input}}; + return std::map{{*part, input}}; } const auto numInput = input->size(); std::map numOfKeys; @@ -66,7 +66,7 @@ std::map KeySelector::partition(const RowVectorPtr& inpu numOfKeys[partition] = index + 1; } - std::map results; + std::map results; for (auto & [key, partitionSize] : numOfKeys) { auto partitionData = wrapChildren(input, partitionSize, keyToIndexBuffers[key]); results[key] = partitionData; diff --git a/velox/experimental/stateful/KeySelector.h b/velox/experimental/stateful/KeySelector.h index baf34967c29a..a9328b3ee9ec 100644 --- a/velox/experimental/stateful/KeySelector.h +++ b/velox/experimental/stateful/KeySelector.h @@ -31,7 +31,7 @@ class KeySelector { memory::MemoryPool* pool, int numPartitions = INT_MAX); - std::map partition(const RowVectorPtr& input); + std::map partition(const RowVectorPtr& input); private: void prepareForInput(const RowVectorPtr& input); diff --git a/velox/experimental/stateful/LocalWindowAggregator.cpp b/velox/experimental/stateful/LocalWindowAggregator.cpp index f99ac04ab015..6c55a308fee7 100644 --- a/velox/experimental/stateful/LocalWindowAggregator.cpp +++ b/velox/experimental/stateful/LocalWindowAggregator.cpp @@ -45,9 +45,9 @@ void LocalWindowAggregator::getOutput() { return; } - std::map keyToData = keySelector_->partition(input_); + std::map keyToData = keySelector_->partition(input_); for (const auto& [key, data] : keyToData) { - std::map sliceEndToData = + std::map sliceEndToData = sliceAssigner_->partition(data); for (const auto& [sliceEnd, data] : sliceEndToData) { // TODO: addElement may have data output. diff --git a/velox/experimental/stateful/ProcessingTimeService.h b/velox/experimental/stateful/ProcessingTimeService.h index aef7e3fff068..eb9d3a167de1 100644 --- a/velox/experimental/stateful/ProcessingTimeService.h +++ b/velox/experimental/stateful/ProcessingTimeService.h @@ -15,46 +15,93 @@ */ #pragma once -#include "velox/type/Timestamp.h" -#include -#include +#include "velox/experimental/stateful/window/TimeWindowUtil.h" +#include #include +#include namespace facebook::velox::stateful { using ProcessingTimeCallback = std::function; +class ProcessingTimerTask { +public: + ProcessingTimerTask( + long time, + ProcessingTimeCallback callback) + : time_(time), callback_(callback) {} + + void operator()() const { + callback_(time_); + } +private: + long time_; + ProcessingTimeCallback callback_; +}; + class ProcessingTimeSerivice { public: long getCurrentProcessingTime() { - return Timestamp::now().toMillis(); + return TimeWindowUtil::getCurrentProcessingTime(); + } + virtual std::optional registerTimer(long timestamp, ProcessingTimerTask target) { + std::optional task; + return task; + } + virtual void cancel(const std::string& task) {} + virtual void close() {} + + void finish(const std::string& task) { + auto it = std::find(registry.begin(), registry.end(), task); + if (it != registry.end()) { + registry.erase(it); + } } - virtual folly::Future registerTimer(long timestamp, ProcessingTimeCallback target) { - return folly::Future::makeEmpty(); + std::string generateTimerTaskName(long timestamp) { + return "task_" + std::to_string(timestamp); } +protected: + std::vector registry; }; class SystemProcessingTimeService : public ProcessingTimeSerivice { public: SystemProcessingTimeService() : ProcessingTimeSerivice() { - executor_ = std::make_shared(1); + executor_ = std::make_shared(); + executor_->start(); } - folly::Future registerTimer(long timestamp, ProcessingTimeCallback callback) override { + std::optional registerTimer(long timestamp, ProcessingTimerTask task) override { long currentTimestamp = getCurrentProcessingTime(); long delay = 0; if (timestamp >= currentTimestamp) { - delay = timestamp - currentTimestamp + 1; + delay = timestamp - currentTimestamp; + } + std::string taskName = generateTimerTaskName(timestamp); + if (delay > 0) { + executor_->addFunctionOnce(task, taskName, std::chrono::microseconds(delay * 1000)); + registry.emplace_back(taskName); + } else { + task(); } - return folly::futures::sleep(std::chrono::microseconds(delay * 1000)) - .via(executor_.get()) - .thenValue([&](auto) { - callback(timestamp); - }); + return std::make_optional(taskName); } -private: - std::shared_ptr executor_; + void cancel(const std::string& task) override { + auto it = std::find(registry.begin(), registry.end(), task); + if (it != registry.end()) { + executor_->cancelFunction(task); + registry.erase(it); + } + } + + void close() override { + if (executor_) { + executor_->shutdown(); + } + } +private: + std::shared_ptr executor_; }; -} \ No newline at end of file +} diff --git a/velox/experimental/stateful/StatefulPlanNode.cpp b/velox/experimental/stateful/StatefulPlanNode.cpp index 8b96a7cb7259..e708c7fc5af0 100644 --- a/velox/experimental/stateful/StatefulPlanNode.cpp +++ b/velox/experimental/stateful/StatefulPlanNode.cpp @@ -250,6 +250,8 @@ folly::dynamic StreamWindowAggregationNode::serialize() const { obj["outputType"] = outputType_->serialize(); obj["isEventTime"] = isEventTime_; obj["rowtimeIndex"] = rowtimeIndex_; + obj["windowStartIndex"] = windowStartIndex_; + obj["windowEndIndex"] = windowEndIndex_; return obj; } @@ -292,7 +294,9 @@ core::PlanNodePtr StreamWindowAggregationNode::create(const folly::dynamic& obj, obj["windowType"].asInt(), outputType, obj["isEventTime"].asBool(), - obj["rowtimeIndex"].asInt()); + obj["rowtimeIndex"].asInt(), + obj["windowStartIndex"].asInt(), + obj["windowEndIndex"].asInt()); } const std::vector& GroupWindowAggsHandlerNode::sources() const { diff --git a/velox/experimental/stateful/StatefulPlanNode.h b/velox/experimental/stateful/StatefulPlanNode.h index e36fa4af5e24..67915209dac6 100644 --- a/velox/experimental/stateful/StatefulPlanNode.h +++ b/velox/experimental/stateful/StatefulPlanNode.h @@ -334,7 +334,9 @@ class StreamWindowAggregationNode : public core::PlanNode { int windowType, const RowTypePtr& outputType, bool isEventTime, - int rowtimeIndex) : + int rowtimeIndex, + int windowStartIndex, + int windowEndIndex) : PlanNode(id), aggregation_(std::move(aggregationNode)), localAgg_(std::move(localAgg)), @@ -349,7 +351,9 @@ class StreamWindowAggregationNode : public core::PlanNode { windowType_(windowType), outputType_(std::move(outputType)), isEventTime_(isEventTime), - rowtimeIndex_(rowtimeIndex) {} + rowtimeIndex_(rowtimeIndex), + windowStartIndex_(windowStartIndex), + windowEndIndex_(windowEndIndex) {} const RowTypePtr& outputType() const override { return outputType_; @@ -407,6 +411,14 @@ class StreamWindowAggregationNode : public core::PlanNode { return rowtimeIndex_; } + int windowStartIndex() const { + return windowStartIndex_; + } + + int windowEndIndex() const { + return windowEndIndex_; + } + const std::vector& sources() const override; std::string_view name() const override { @@ -434,6 +446,8 @@ class StreamWindowAggregationNode : public core::PlanNode { const RowTypePtr outputType_; bool isEventTime_; int rowtimeIndex_; + int windowStartIndex_; + int windowEndIndex_; }; class GroupWindowAggsHandlerNode : public core::PlanNode { diff --git a/velox/experimental/stateful/StatefulPlanner.cpp b/velox/experimental/stateful/StatefulPlanner.cpp index fc4bce695ac7..189d4b7431ac 100644 --- a/velox/experimental/stateful/StatefulPlanner.cpp +++ b/velox/experimental/stateful/StatefulPlanner.cpp @@ -189,7 +189,9 @@ StatefulOperatorPtr StatefulPlanner::nodeToStatefulOperator( std::move(globalSliceAssigner), windowAggNode->windowInterval(), windowAggNode->useDayLightSaving(), - windowAggNode->isEventTime()); + windowAggNode->isEventTime(), + windowAggNode->windowStartIndex(), + windowAggNode->windowEndIndex()); } } else if ( auto windowAggNode = @@ -312,7 +314,7 @@ std::unique_ptr StatefulPlanner::nodeToOperator( if (aggregationNode->isPreGrouped()) { return std::make_unique(opId.fetch_add(1), ctx, aggregationNode); } else { - return std::make_unique(opId.fetch_add(1), ctx, aggregationNode); + return std::make_unique(opId.fetch_add(1), ctx, aggregationNode); } } else if ( auto expandNode = diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 0c8cca9f2715..9dcf81f9945e 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -15,9 +15,13 @@ */ #include "velox/experimental/stateful/WindowAggregator.h" #include +#include +#include #include "velox/experimental/stateful/window/TimeWindowUtil.h" #include +#include +#include namespace facebook::velox::stateful { @@ -29,20 +33,26 @@ WindowAggregator::WindowAggregator( std::unique_ptr sliceAssigner, const long windowInterval, const bool useDayLightSaving, - const bool isEventTime) + const bool isEventTime, + const int windowStartIndex, + const int windowEndIndex) : StatefulOperator(std::move(globalAggerator), std::move(targets)), localAggerator_(std::move(localAggerator)), keySelector_(std::move(keySelector)), sliceAssigner_(std::move(sliceAssigner)), windowInterval_(windowInterval), useDayLightSaving_(useDayLightSaving), - isEventTime_(isEventTime) { + isEventTime_(isEventTime), + windowStartIndex_(windowStartIndex), + windowEndIndex_(windowEndIndex) { windowBuffer_ = std::make_shared(); } void WindowAggregator::initialize() { StatefulOperator::initialize(); - localAggerator_->initialize(); + if (localAggerator_) { + localAggerator_->initialize(); + } StateDescriptor stateDesc("window-aggs"); windowState_ = stateHandler()->getValueState(stateDesc); @@ -51,22 +61,25 @@ void WindowAggregator::initialize() { void WindowAggregator::addInput(RowVectorPtr input) { VELOX_CHECK(!input_, "Last input has not been processed"); + LOG(INFO) << "add input here"; input_ = input; } void WindowAggregator::getOutput() { + std::cout << "getOutput 111:" << op()->needsInput() << std::endl; if (!input_) { return; } - std::map keyToData = keySelector_->partition(input_); + std::map keyToData = keySelector_->partition(input_); for (const auto& [key, data] : keyToData) { - std::map sliceEndToData = + std::map sliceEndToData = sliceAssigner_->assignSliceEnd(data); for (const auto& [sliceEnd, data] : sliceEndToData) { auto windowData = data; if (!isEventTime_) { - windowTimerService_->registerProcessingTimeTimer(key, sliceEnd, sliceEnd - 1); + std::cout << "register process time:" << sliceEnd << std::endl; + windowTimerService_->registerProcessingTimeTimer(key, sliceEnd, sliceEnd); } if (isEventTime_ && TimeWindowUtil::isWindowFired(sliceEnd, currentProgress_, shiftTimeZone_)) { @@ -90,6 +103,7 @@ void WindowAggregator::getOutput() { } } else { // the assigned slice hasn't been triggered, accumulate into the assigned slice + LOG(INFO) << "windowData.size:" << windowData->size() << " sliceEnd:" << sliceEnd; windowBuffer_->addElement(key, sliceEnd, windowData); } } @@ -137,10 +151,91 @@ void WindowAggregator::processWatermarkInternal(long timestamp) { } } +/// Add window_start / window_end timestamp to output +RowVectorPtr addWindowTimestampToOutput( + const RowVectorPtr& output, + const std::string& fieldName, + const TypePtr& fieldType, + const long fieldValue, + const int fieldIndex) { + auto createTimestampVector = [&]( + const Timestamp& val, + const size_t size, + velox::memory::MemoryPool* pool) -> VectorPtr { + const TypePtr windowStartType = std::make_shared(); + VectorPtr windowStartVec = BaseVector::create(windowStartType, output->size(), output->pool()); + FlatVector* timestampVector = windowStartVec->asFlatVector(); + for (size_t i = 0; i < size; ++i) { + timestampVector->set(i, val); + } + return windowStartVec; + }; + const TypePtr& outputType = output->type(); + const RowTypePtr& outputRowType = std::dynamic_pointer_cast(outputType); + const std::vector& outputFieldNames = outputRowType->names(); + const std::vector& outputFieldTypes = outputRowType->children(); + const std::vector& outputFields = output->children(); + std::vector newOutputFieldNames; + std::vector newOutputFieldTypes; + std::vector newOutputFields; + VectorPtr windowStartVec = createTimestampVector(Timestamp::fromMillis(fieldValue), output->size(), output->pool()); + for (int i = 0; i < fieldIndex; ++i) { + newOutputFieldTypes.emplace_back(outputFieldTypes[i]); + newOutputFieldNames.emplace_back(outputFieldNames[i]); + newOutputFields.emplace_back(outputFields[i]); + } + newOutputFieldTypes.emplace_back(fieldType); + newOutputFieldNames.emplace_back(fieldName); + newOutputFields.emplace_back(windowStartVec); + for (int i = fieldIndex + 1; i < output->childrenSize() + 1; ++i) { + newOutputFieldTypes.emplace_back(outputFieldTypes[i-1]); + newOutputFieldNames.emplace_back(outputFieldNames[i-1]); + newOutputFields.emplace_back(outputFields[i-1]); + } + auto newOutputRowType = std::make_shared(std::move(newOutputFieldNames), std::move(newOutputFieldTypes)); + return std::make_shared(output->pool(), + newOutputRowType, + output->nulls(), + output->size(), + newOutputFields, + output->getNullCount() + ); +} + void WindowAggregator::onTimer(std::shared_ptr> timer) { - auto output = windowState_->value(timer->key(), timer->ns()); - windowState_->remove(timer->key(), timer->ns()); - pushOutput(output); + fireWindow(timer->key(), timer->timestamp(), timer->ns()); + clearWindow(timer->key(), timer->timestamp(), timer->ns()); +} + +template +void WindowAggregator::fireWindow(K key, long timerTimestamp, long windowEnd) { + RowVectorPtr output = windowState_->value(key, windowEnd); + if (output) { + if (windowStartIndex_ >= 0) { + output = addWindowTimestampToOutput( + output, + "window_start", + std::make_shared(), + windowEnd - windowInterval_, + windowStartIndex_); + } + if (windowEndIndex_ >= 0) { + output = addWindowTimestampToOutput( + output, + "window_end", + std::make_shared(), + windowEnd, + windowEndIndex_); + } + } + if (output) { + pushOutput(output); + } +} + +template +void WindowAggregator::clearWindow(K key, long timerTimestamp, long windowEnd) { + windowState_->remove(key, windowEnd); } void WindowAggregator::onEventTime(std::shared_ptr> timer) { @@ -148,6 +243,7 @@ void WindowAggregator::onEventTime(std::shared_ptr> timer) { + LOG(INFO) << "window agg on processing Time:" << lastTriggeredProcessingTime_ << " timer->timestamp():" << timer->timestamp(); if (timer->timestamp() > lastTriggeredProcessingTime_) { lastTriggeredProcessingTime_ = timer->timestamp(); auto windowKeyToData = windowBuffer_->advanceProgress(timer->timestamp()); @@ -163,12 +259,16 @@ void WindowAggregator::onProcessingTime(std::shared_ptraddInput(TimeWindowUtil::mergeVectors(allDatas, op()->pool())); + RowVectorPtr opInput = TimeWindowUtil::mergeVectors(allDatas, op()->pool()); + LOG(INFO) << "opInput:" << opInput->size() << "data:" << opInput->toString(0); + op()->addInput(opInput); auto newAcc = op()->getOutput(); if (newAcc) { + LOG(INFO) << "newAcc:" << newAcc->size() << " newAcc.data:" << newAcc->toString(0); windowState_->update(windowKey.key(), windowKey.window(), newAcc); } } + windowBuffer_->clear(); onTimer(timer); } } @@ -181,7 +281,9 @@ long WindowAggregator::sliceStateMergeTarget(long sliceToMerge) { void WindowAggregator::close() { processWatermarkInternal(INT_MAX); StatefulOperator::close(); - localAggerator_->close(); + if (localAggerator_) { + localAggerator_->close(); + } input_.reset(); windowBuffer_->clear(); windowState_->clear(); diff --git a/velox/experimental/stateful/WindowAggregator.h b/velox/experimental/stateful/WindowAggregator.h index 72ce565b3fc2..b06e65c46384 100644 --- a/velox/experimental/stateful/WindowAggregator.h +++ b/velox/experimental/stateful/WindowAggregator.h @@ -15,6 +15,12 @@ */ #pragma once +#include +#include +#include +#include +#include +#include #include "velox/experimental/stateful/InternalTimerService.h" #include "velox/experimental/stateful/KeySelector.h" #include "velox/experimental/stateful/StatefulOperator.h" @@ -37,7 +43,9 @@ class WindowAggregator : public StatefulOperator, public Triggerable sliceAssigner, const long windowInterval, const bool useDayLightSaving, - const bool isEventTime); + const bool isEventTime, + const int windowStartIndex, + const int windowEndIndex); void initialize() override; @@ -62,6 +70,12 @@ class WindowAggregator : public StatefulOperator, public Triggerable> timer); + template + void fireWindow(K key, long timerTImestamp, long windowEnd); + + template + void clearWindow(K k, long timerTimestamp, long windowEnd); + std::unique_ptr localAggerator_; std::unique_ptr keySelector_; std::unique_ptr sliceAssigner_; @@ -70,6 +84,8 @@ class WindowAggregator : public StatefulOperator, public Triggerable> windowState_; std::shared_ptr> windowTimerService_; }; + } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/window/MergingWindowSet.h b/velox/experimental/stateful/window/MergingWindowSet.h index 3326ea9561d8..ab727993bcad 100644 --- a/velox/experimental/stateful/window/MergingWindowSet.h +++ b/velox/experimental/stateful/window/MergingWindowSet.h @@ -19,7 +19,6 @@ #include "velox/experimental/stateful/window/WindowProcessFunction.h" #include "velox/experimental/stateful/window/Window.h" #include -#include #include namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/window/SliceAssigner.cpp b/velox/experimental/stateful/window/SliceAssigner.cpp index c831a1fc5585..d73e63ed4d97 100644 --- a/velox/experimental/stateful/window/SliceAssigner.cpp +++ b/velox/experimental/stateful/window/SliceAssigner.cpp @@ -14,6 +14,7 @@ * limitations under the License. */ #include "velox/experimental/stateful/window/SliceAssigner.h" +#include "velox/experimental/stateful/window/TimeWindowUtil.h" #include #include @@ -38,13 +39,19 @@ SliceAssigner::SliceAssigner( sliceSize_ = std::gcd(size, step); } -std::map SliceAssigner::assignSliceEnd(const RowVectorPtr& input) { +std::map SliceAssigner::assignSliceEnd(const RowVectorPtr& input) { if (rowtimeIndex_ < 0) { // TODO: using Processing Time Service - auto now = std::chrono::system_clock::now(); - long timestamp_ms = std::chrono::duration_cast( - now.time_since_epoch()).count(); - return {{timestamp_ms, input}}; + // TODO: using TimeWindow.getWindowStartWithOffset to get window start, start + size as end + long timestamp_ms = TimeWindowUtil::getCurrentProcessingTime(); + if (windowType_ == 1) { // tumble window + // TODO:: support get utcTimestamp by timezone. + long utcTimestamp = TimeWindowUtil::toEpochMillsForTimer(timestamp_ms, 0); + long windowStart = stateful::TimeWindowUtil::getWindowStartWithOffset(utcTimestamp, offset_, size_); + return {{windowStart + size_, input}}; + } else { + return {{timestamp_ms, input}}; + } } return keySelector_->partition(input); } diff --git a/velox/experimental/stateful/window/SliceAssigner.h b/velox/experimental/stateful/window/SliceAssigner.h index a46bba22575e..5d2ac294c869 100644 --- a/velox/experimental/stateful/window/SliceAssigner.h +++ b/velox/experimental/stateful/window/SliceAssigner.h @@ -30,7 +30,7 @@ class SliceAssigner { int windowType, int rowtimeIndex); - std::map assignSliceEnd(const RowVectorPtr& input); + std::map assignSliceEnd(const RowVectorPtr& input); long getLastWindowEnd(long sliceEnd); diff --git a/velox/experimental/stateful/window/TimeWindowUtil.cpp b/velox/experimental/stateful/window/TimeWindowUtil.cpp index d6e1a25b7713..d2c9c376228c 100644 --- a/velox/experimental/stateful/window/TimeWindowUtil.cpp +++ b/velox/experimental/stateful/window/TimeWindowUtil.cpp @@ -60,6 +60,11 @@ long TimeWindowUtil::getWindowStartWithOffset(long timestamp, long offset, long } } +long TimeWindowUtil::getCurrentProcessingTime() { + auto now = std::chrono::system_clock::now(); + return std::chrono::duration_cast(now.time_since_epoch()).count(); +} + // static bool TimeWindowUtil::isWindowFired( long windowEnd, long currentProgress, int shiftTimeZone) { diff --git a/velox/experimental/stateful/window/TimeWindowUtil.h b/velox/experimental/stateful/window/TimeWindowUtil.h index 54ef2e69383b..3cb1fabb3c88 100644 --- a/velox/experimental/stateful/window/TimeWindowUtil.h +++ b/velox/experimental/stateful/window/TimeWindowUtil.h @@ -43,6 +43,8 @@ class TimeWindowUtil { static long toEpochMillsForTimer(long timestamp, int shiftTimeZone); static long cleanupTime(long maxTimestamp, long allowedLateness_, bool isEventTime); + + static long getCurrentProcessingTime(); }; From 77a93ef8a13077d4ec4000f73fe04dba020cb703 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Wed, 15 Oct 2025 09:52:18 +0000 Subject: [PATCH 03/15] support proctime window --- velox/exec/StreamingAggregation.cpp | 8 +++---- .../stateful/InternalTimerService.h | 3 +-- .../stateful/ProcessingTimeService.h | 2 +- .../stateful/WindowAggregator.cpp | 21 +++++++------------ velox/experimental/stateful/state/State.h | 2 +- .../stateful/window/WindowBuffer.h | 2 +- 6 files changed, 15 insertions(+), 23 deletions(-) diff --git a/velox/exec/StreamingAggregation.cpp b/velox/exec/StreamingAggregation.cpp index f08729e676e3..8733d6e484fb 100644 --- a/velox/exec/StreamingAggregation.cpp +++ b/velox/exec/StreamingAggregation.cpp @@ -210,10 +210,9 @@ void StreamingAggregation::assignGroups() { auto numInput = input_->size(); inputGroups_.resize(numInput); - // Look for the end of the last group. vector_size_t index = 0; - if (prevInput_) { + if (prevInput_ && numGroups_ > 0) { auto prevIndex = prevInput_->size() - 1; auto* prevGroup = groups_[numGroups_ - 1]; for (; index < numInput; ++index) { @@ -334,10 +333,9 @@ RowVectorPtr StreamingAggregation::getOutput() { evaluateAggregates(); RowVectorPtr output; - - if (numGroups_ > minOutputBatchSize_) { + if (numGroups_ >= minOutputBatchSize_) { output = createOutput( - std::min(numGroups_ - 1, static_cast(maxOutputBatchSize_))); + std::min(numGroups_, static_cast(maxOutputBatchSize_))); } prevInput_ = input_; diff --git a/velox/experimental/stateful/InternalTimerService.h b/velox/experimental/stateful/InternalTimerService.h index 39c3d102737a..019735f57985 100644 --- a/velox/experimental/stateful/InternalTimerService.h +++ b/velox/experimental/stateful/InternalTimerService.h @@ -46,7 +46,6 @@ class InternalTimerService { std::shared_ptr> oldHead = processingTimeTimersQueue_.peek(); processingTimeTimersQueue_.add(std::make_shared>(time, key, ns)); long nextTriggerTime = oldHead != nullptr ? oldHead->timestamp() : std::numeric_limits::max() ; - std::cout << "time:" << time << " nextTriggerTime:" << nextTriggerTime << std::endl; if (time < nextTriggerTime) { if (nextTimer_.has_value()) { processingTimeService_->cancel(nextTimer_.value()); @@ -58,7 +57,7 @@ class InternalTimerService { } void deleteProcessingTimeTimer(K key, N ns, long time) { - eventTimeTimersQueue_.remove(std::make_shared>(time, key, ns)); + processingTimeTimersQueue_.remove(std::make_shared>(time, key, ns)); } long currentWatermark() { diff --git a/velox/experimental/stateful/ProcessingTimeService.h b/velox/experimental/stateful/ProcessingTimeService.h index eb9d3a167de1..b08de25e44ee 100644 --- a/velox/experimental/stateful/ProcessingTimeService.h +++ b/velox/experimental/stateful/ProcessingTimeService.h @@ -59,7 +59,7 @@ class ProcessingTimeSerivice { } std::string generateTimerTaskName(long timestamp) { - return "task_" + std::to_string(timestamp); + return "proc_time_task_" + std::to_string(timestamp); } protected: std::vector registry; diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 9dcf81f9945e..9f68b314d751 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -14,9 +14,9 @@ * limitations under the License. */ #include "velox/experimental/stateful/WindowAggregator.h" -#include -#include -#include +#include "velox/experimental/stateful/TimerHeapInternalTimer.h" +#include "velox/type/Type.h" +#include "velox/vector/ComplexVector.h" #include "velox/experimental/stateful/window/TimeWindowUtil.h" #include @@ -45,8 +45,8 @@ WindowAggregator::WindowAggregator( isEventTime_(isEventTime), windowStartIndex_(windowStartIndex), windowEndIndex_(windowEndIndex) { - windowBuffer_ = std::make_shared(); -} + windowBuffer_ = std::make_shared(); + } void WindowAggregator::initialize() { StatefulOperator::initialize(); @@ -61,12 +61,10 @@ void WindowAggregator::initialize() { void WindowAggregator::addInput(RowVectorPtr input) { VELOX_CHECK(!input_, "Last input has not been processed"); - LOG(INFO) << "add input here"; input_ = input; } void WindowAggregator::getOutput() { - std::cout << "getOutput 111:" << op()->needsInput() << std::endl; if (!input_) { return; } @@ -78,7 +76,7 @@ void WindowAggregator::getOutput() { for (const auto& [sliceEnd, data] : sliceEndToData) { auto windowData = data; if (!isEventTime_) { - std::cout << "register process time:" << sliceEnd << std::endl; + std::cout << "register process time:" << sliceEnd << " key:" << key << std::endl; windowTimerService_->registerProcessingTimeTimer(key, sliceEnd, sliceEnd); } @@ -103,7 +101,6 @@ void WindowAggregator::getOutput() { } } else { // the assigned slice hasn't been triggered, accumulate into the assigned slice - LOG(INFO) << "windowData.size:" << windowData->size() << " sliceEnd:" << sliceEnd; windowBuffer_->addElement(key, sliceEnd, windowData); } } @@ -244,10 +241,10 @@ void WindowAggregator::onEventTime(std::shared_ptr> timer) { LOG(INFO) << "window agg on processing Time:" << lastTriggeredProcessingTime_ << " timer->timestamp():" << timer->timestamp(); - if (timer->timestamp() > lastTriggeredProcessingTime_) { + if (timer->timestamp() >= lastTriggeredProcessingTime_) { lastTriggeredProcessingTime_ = timer->timestamp(); auto windowKeyToData = windowBuffer_->advanceProgress(timer->timestamp()); - for (const auto&[windowKey, datas] : windowKeyToData) { + for (const auto& [windowKey, datas] : windowKeyToData) { if (datas.empty()) { continue; } @@ -260,11 +257,9 @@ void WindowAggregator::onProcessingTime(std::shared_ptrpool()); - LOG(INFO) << "opInput:" << opInput->size() << "data:" << opInput->toString(0); op()->addInput(opInput); auto newAcc = op()->getOutput(); if (newAcc) { - LOG(INFO) << "newAcc:" << newAcc->size() << " newAcc.data:" << newAcc->toString(0); windowState_->update(windowKey.key(), windowKey.window(), newAcc); } } diff --git a/velox/experimental/stateful/state/State.h b/velox/experimental/stateful/state/State.h index 4b3635e11546..34fe10cc0ecf 100644 --- a/velox/experimental/stateful/state/State.h +++ b/velox/experimental/stateful/state/State.h @@ -57,7 +57,7 @@ class ListState : public State { // virtual void update(std::list& values) = 0; }; -// This class is relevent to flink org.apache.flink.api.common.ListState. +// This class is relevent to flink org.apache.flink.api.common.ValueState. template class ValueState : public State { public: diff --git a/velox/experimental/stateful/window/WindowBuffer.h b/velox/experimental/stateful/window/WindowBuffer.h index 20bfd7d5725e..a0ca2ea1f93c 100644 --- a/velox/experimental/stateful/window/WindowBuffer.h +++ b/velox/experimental/stateful/window/WindowBuffer.h @@ -18,7 +18,6 @@ #include "velox/experimental/stateful/window/WindowKey.h" #include "velox/vector/ComplexVector.h" #include -#include namespace facebook::velox::stateful { @@ -53,6 +52,7 @@ class RecordsWindowBuffer : public WindowBuffer { std::unordered_map> buffer_; // This is used to return empty map when no window is fired. std::unordered_map> empty_; + long minSliceEnd_ = LONG_MAX; int shiftTimeZone_ = 0; // TODO: support time zone shift }; From b4f7e4aa1d280c715edd1222ae01c5337051d896 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Wed, 15 Oct 2025 11:54:09 +0000 Subject: [PATCH 04/15] remove useless changes --- velox/connectors/kafka/CMakeLists.txt | 40 ---- velox/connectors/kafka/KafkaConfig.cpp | 111 ----------- velox/connectors/kafka/KafkaConfig.h | 146 -------------- velox/connectors/kafka/KafkaConnector.cpp | 42 ---- velox/connectors/kafka/KafkaConnector.h | 81 -------- .../connectors/kafka/KafkaConnectorSplit.cpp | 113 ----------- velox/connectors/kafka/KafkaConnectorSplit.h | 81 -------- velox/connectors/kafka/KafkaConsumer.cpp | 111 ----------- velox/connectors/kafka/KafkaConsumer.h | 70 ------- velox/connectors/kafka/KafkaDataSource.cpp | 165 ---------------- velox/connectors/kafka/KafkaDataSource.h | 118 ----------- velox/connectors/kafka/KafkaTableHandle.cpp | 86 -------- velox/connectors/kafka/KafkaTableHandle.h | 76 -------- velox/connectors/kafka/format/CMakeLists.txt | 21 -- .../kafka/format/CSVRecordDeserializer.h | 37 ---- .../kafka/format/JSONRecordDeserializer.cpp | 103 ---------- .../kafka/format/JSONRecordDeserializer.h | 160 --------------- .../kafka/format/KafkaRecordDeserializer.h | 55 ------ .../kafka/format/RawRecordDeserializer.h | 56 ------ .../format/StreamJSONRecordDeserializer.cpp | 71 ------- .../format/StreamJSONRecordDeserializer.h | 165 ---------------- velox/connectors/kafka/tests/CMakeLists.txt | 27 --- .../kafka/tests/KafkaConnectorTest.cpp | 163 ---------------- .../kafka/tests/KafkaConnectorTestBase.h | 184 ------------------ velox/connectors/kafka/tests/kafka.conf | 2 - .../stateful/WindowAggregator.cpp | 6 +- .../experimental/stateful/WindowAggregator.h | 9 +- .../stateful/window/MergingWindowSet.h | 1 + .../stateful/window/WindowBuffer.h | 2 +- 29 files changed, 7 insertions(+), 2295 deletions(-) delete mode 100644 velox/connectors/kafka/CMakeLists.txt delete mode 100644 velox/connectors/kafka/KafkaConfig.cpp delete mode 100644 velox/connectors/kafka/KafkaConfig.h delete mode 100644 velox/connectors/kafka/KafkaConnector.cpp delete mode 100644 velox/connectors/kafka/KafkaConnector.h delete mode 100644 velox/connectors/kafka/KafkaConnectorSplit.cpp delete mode 100644 velox/connectors/kafka/KafkaConnectorSplit.h delete mode 100644 velox/connectors/kafka/KafkaConsumer.cpp delete mode 100644 velox/connectors/kafka/KafkaConsumer.h delete mode 100644 velox/connectors/kafka/KafkaDataSource.cpp delete mode 100644 velox/connectors/kafka/KafkaDataSource.h delete mode 100644 velox/connectors/kafka/KafkaTableHandle.cpp delete mode 100644 velox/connectors/kafka/KafkaTableHandle.h delete mode 100644 velox/connectors/kafka/format/CMakeLists.txt delete mode 100644 velox/connectors/kafka/format/CSVRecordDeserializer.h delete mode 100644 velox/connectors/kafka/format/JSONRecordDeserializer.cpp delete mode 100644 velox/connectors/kafka/format/JSONRecordDeserializer.h delete mode 100644 velox/connectors/kafka/format/KafkaRecordDeserializer.h delete mode 100644 velox/connectors/kafka/format/RawRecordDeserializer.h delete mode 100644 velox/connectors/kafka/format/StreamJSONRecordDeserializer.cpp delete mode 100644 velox/connectors/kafka/format/StreamJSONRecordDeserializer.h delete mode 100644 velox/connectors/kafka/tests/CMakeLists.txt delete mode 100644 velox/connectors/kafka/tests/KafkaConnectorTest.cpp delete mode 100644 velox/connectors/kafka/tests/KafkaConnectorTestBase.h delete mode 100644 velox/connectors/kafka/tests/kafka.conf diff --git a/velox/connectors/kafka/CMakeLists.txt b/velox/connectors/kafka/CMakeLists.txt deleted file mode 100644 index 24f61712c4d4..000000000000 --- a/velox/connectors/kafka/CMakeLists.txt +++ /dev/null @@ -1,40 +0,0 @@ -# Copyright (c) Facebook, Inc. and its affiliates. -# -# Licensed 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. - -# velox_add_library(velox_kafka_config OBJECT KafkaConfig.cpp) -# velox_link_libraries(velox_kafka_config velox_core velox_exception) - -add_subdirectory(format) - -velox_add_library( - velox_kafka_connector - OBJECT - KafkaConfig.cpp - KafkaConsumer.cpp - KafkaTableHandle.cpp - KafkaDataSource.cpp - KafkaConnector.cpp - KafkaConnectorSplit.cpp) - -velox_link_libraries( - velox_kafka_connector - velox_kafka_connector_format - velox_common_io - velox_connector - Folly::folly - CppKafka::cppkafka) - -if(${VELOX_BUILD_TESTING}) - add_subdirectory(tests) -endif() diff --git a/velox/connectors/kafka/KafkaConfig.cpp b/velox/connectors/kafka/KafkaConfig.cpp deleted file mode 100644 index 16ec3df6eebf..000000000000 --- a/velox/connectors/kafka/KafkaConfig.cpp +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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 "velox/connectors/kafka/KafkaConfig.h" - -namespace facebook::velox::connector::kafka { - -template -const T KafkaConfig::checkAndGetConfigValue( - const std::string& configKey, - T defaultValue) const { - std::optional configValue = - static_cast>(config_->get(configKey)); - if constexpr (throwException) { - VELOX_CHECK_EQ( - configValue.has_value(), - true, - "Kafka config {} has no specified value.", - configKey); - } - if (configValue.has_value()) { - return configValue.value(); - } else { - return defaultValue; - } -} - -const std::string ConnectionConfig::getBootstrapServers() const { - return checkAndGetConfigValue(kBootstrapServers, ""); -} - -const std::string ConnectionConfig::getTopic() const { - return checkAndGetConfigValue(kTopic, ""); -} - -const std::string ConnectionConfig::getGroupId() const { - return checkAndGetConfigValue(kGroupId, ""); -} - -const std::string ConnectionConfig::getClientId() const { - return checkAndGetConfigValue(kClientId, ""); -} - -const std::string ConnectionConfig::getFormat() const { - return checkAndGetConfigValue(kFormat, ""); -} - -const std::string ConnectionConfig::getAutoOffsetReset() const { - return checkAndGetConfigValue( - kAutoResetOffset, defaultAutoOffsetRest); -} - -const uint32_t ConnectionConfig::getQueuedMinMessages() const { - return checkAndGetConfigValue( - kQueueMinMessages, defaultQueuedMinMessages); -} - -const bool ConnectionConfig::getEnableAutoCommit() const { - return checkAndGetConfigValue(kEnableAutoCommit, "true") == - "true" - ? true - : false; -} - -const bool ConnectionConfig::getEnablePartitionEof() const { - return checkAndGetConfigValue(kEnablePartitionEof, false); -} - -const uint32_t ConnectionConfig::getDataBatchSize() const { - return checkAndGetConfigValue( - kDataBatchSize, defaultDataBatchSize); -} - -const uint32_t ConnectionConfig::getPollTimeoutMills() const { - return checkAndGetConfigValue( - kPollTimeoutMills, defaultPollTimeoutMills); -} - -const std::string ConnectionConfig::getStartupMode() const { - return checkAndGetConfigValue( - kStartupMode, defaultConsumeStartupMode); -} - -cppkafka::Configuration ConnectionConfig::getCppKafkaConfiguration() const { - cppkafka::Configuration conf; - conf.set("metadata.broker.list", getBootstrapServers()); - conf.set("group.id", getGroupId()); - conf.set("client.id", getClientId()); - conf.set("client.software.name", defaultClientSoftwareName); - conf.set("client.software.version", defaultClientSoftwareVersion); - conf.set("auto.offset.reset", getAutoOffsetReset()); - conf.set("queued.min.messages", getQueuedMinMessages()); - conf.set("enable.auto.commit", getEnableAutoCommit()); - conf.set("auto.commit.interval.ms", 2000); - conf.set("enable.partition.eof", getEnablePartitionEof()); - return conf; -} -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConfig.h b/velox/connectors/kafka/KafkaConfig.h deleted file mode 100644 index 2a2009882b66..000000000000 --- a/velox/connectors/kafka/KafkaConfig.h +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ -#pragma once - -#include "velox/common/config/Config.h" -#include - -namespace facebook::velox::connector::kafka { - -using ConfigPtr = std::shared_ptr; - -/// Kafka config base class. -class KafkaConfig { - public: - KafkaConfig(const ConfigPtr& config) : config_(config) {} - - const ConfigPtr& getConfig() const { - return config_; - } - - const bool exists(const std::string& configKey) const { - return config_ && config_->valueExists(configKey); - } - - const bool empty() const { - if (!config_) { - return true; - } else { - return config_->rawConfigs().empty(); - } - } - - template - const std::shared_ptr setConfigs( - const std::unordered_map& configs) const { - std::unordered_map rawConfigs = config_->rawConfigsCopy(); - rawConfigs.insert(configs.begin(), configs.end()); - ConfigPtr newConfig = - std::make_shared(std::move(rawConfigs)); - return std::make_shared(newConfig); - } - - protected: - ConfigPtr config_; - template - const T checkAndGetConfigValue(const std::string& configKey, T defaultValue) const; -}; - -/// Kafka connector config. -class ConnectionConfig : public KafkaConfig { - public: - ConnectionConfig(const ConfigPtr& config) : KafkaConfig(config) {} - /// The config key of bootstrap servers - static constexpr const char* kBootstrapServers = "bootstrap.servers"; - /// The config key of topic - static constexpr const char* kTopic = "topic"; - /// The config key of group id - static constexpr const char* kGroupId = "group.id"; - /// The config key of client id - static constexpr const char* kClientId = "client.id"; - /// The config key fo format - static constexpr const char* kFormat = "format"; - /// The config key of auto offset reset - static constexpr const char* kAutoResetOffset = "auto.offset.reset"; - /// The config key of minimum number of messages of the queue buffer - static constexpr const char* kQueueMinMessages = "queued.min.messages"; - /// The config key of whether to enable auto commit kafka coffset - static constexpr const char* kEnableAutoCommit = "enable.auto.commit"; - /// The config key of whether to ignore partition eof - static constexpr const char* kEnablePartitionEof = "enable.partition.eof"; - /// The config key of max batch size to poll kafka messages. - static constexpr const char* kDataBatchSize = "data.batch.size"; - /// The config key of timeout milliseconds to poll kafka messages. - static constexpr const char* kPollTimeoutMills = "poll.timeout.mills"; - /// The config key of queue buffer size of cppkafka client - static constexpr const char* kConsumeMessageQueueSize = "consume.queue.size"; - /// The startup mode of kafka consumer, its value canbe `group-offsets`, - /// `latest-offsets`, `earliest-offsets`, `timestamp`. - static constexpr const char* kStartupMode = "scan.startup.mode"; - /// The config of kafka client, to define the default value of minimum - /// messages size of kafka client queue. - static constexpr const uint32_t defaultQueuedMinMessages = 1000000; - /// The config of the kafka client, to define the default software name of - /// client. - static constexpr const char* defaultClientSoftwareName = "velox"; - /// The config of the kafka client, to define the default version of client. - static constexpr const char* defaultClientSoftwareVersion = "***"; - /// Define the default batch size of a data process. - static constexpr const uint32_t defaultDataBatchSize = 500; - /// Define the default poll batch size of kafka client. - static constexpr const uint32_t defaultPollBatchSize = 500; - /// The config of the kafka client, to define the default timeout millseconds - /// of a single consumption. - static constexpr const uint32_t defaultPollTimeoutMills = 100; - /// The config of kafka client, to define the default value of config - /// `auto.offset.reset` - static constexpr const char* defaultAutoOffsetRest = "latest"; - /// The config of kafka client, to define the default value of config - /// `scan.startup.mode` - static constexpr const char* defaultConsumeStartupMode = "group-offsets"; - - const std::string getBootstrapServers() const; - const std::string getTopic() const; - const std::string getGroupId() const; - const std::string getClientId() const; - const std::string getFormat() const; - const std::string getAutoOffsetReset() const; - const uint32_t getQueuedMinMessages() const; - const bool getEnableAutoCommit() const; - const bool getEnablePartitionEof() const; - const uint32_t getDataBatchSize() const; - const uint32_t getPollTimeoutMills() const; - const uint32_t getConsumeQueueSize() const; - const std::string getStartupMode() const; - /// Get the configuration for kafka client to consume. - cppkafka::Configuration getCppKafkaConfiguration() const; -}; - -/// The json format config for kafka. -class JSONFormatConfig : public KafkaConfig { - public: - JSONFormatConfig(const ConfigPtr& config) : KafkaConfig(config) {} -}; - -/// The csv format config for kafka. -class CSVFormatConfig : public KafkaConfig {}; - -using KafkaConfigPtr = std::shared_ptr; -using ConnectionConfigPtr = std::shared_ptr; -using JSONFormatConfigPtr = std::shared_ptr; -using CSVFormatConfigPtr = std::shared_ptr; - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConnector.cpp b/velox/connectors/kafka/KafkaConnector.cpp deleted file mode 100644 index 71b92dd434ea..000000000000 --- a/velox/connectors/kafka/KafkaConnector.cpp +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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 "velox/connectors/kafka/KafkaConnector.h" -#include "velox/connectors/kafka/KafkaDataSource.h" - -namespace facebook::velox::connector::kafka { - -std::unique_ptr KafkaConnector::createDataSource( - const RowTypePtr& outputType, - const ConnectorHandlePtr& tableHandle, - const std::unordered_map>& /* columnHandles **/, - ConnectorQueryCtx* connectorQueryCtx) { - return std::make_unique( - outputType, - tableHandle, - connectorQueryCtx, - config_); -} - -std::unique_ptr KafkaConnector::createDataSink( - RowTypePtr inputType, - ConnectorInsertTableHandlePtr connectorInsertTableHandle, - ConnectorQueryCtx* connectorQueryCtx, - CommitStrategy commitStrategy) { - VELOX_NYI(); -} - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConnector.h b/velox/connectors/kafka/KafkaConnector.h deleted file mode 100644 index 04a1f22aadca..000000000000 --- a/velox/connectors/kafka/KafkaConnector.h +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ -#pragma once - -#include "velox/connectors/Connector.h" -#include "velox/connectors/kafka/KafkaConfig.h" -#include "velox/connectors/kafka/KafkaDataSource.h" - -namespace facebook::velox::connector::kafka { - -using ConnectorHandlePtr = std::shared_ptr; -using ConnectorInsertTableHandlePtr = - std::shared_ptr; - -/// The kafka connector. -class KafkaConnector : public Connector { - public: - KafkaConnector( - const std::string& id, - std::shared_ptr config, - folly::Executor* /* executor **/) - : Connector(id), - config_(std::make_shared(config)) {} - - const std::shared_ptr& connectorConfig() - const override { - return config_->getConfig(); - } - - ConnectorMetadata* metadata() const override { - VELOX_NYI(); - } - - std::unique_ptr createDataSource( - const RowTypePtr& outputType, - const ConnectorHandlePtr& tableHandle, - const std::unordered_map>& columnHandles, - ConnectorQueryCtx* connectorQueryCtx) override; - - std::unique_ptr createDataSink( - RowTypePtr inputType, - ConnectorInsertTableHandlePtr connectorInsertTableHandle, - ConnectorQueryCtx* connectorQueryCtx, - CommitStrategy commitStrategy) override; - - private: - const ConnectionConfigPtr config_; -}; - -class KafkaConnectorFactory : public ConnectorFactory { - public: - static constexpr const char* kKafkaConnectorName{"Kafka"}; - - KafkaConnectorFactory() : ConnectorFactory(kKafkaConnectorName) {} - - explicit KafkaConnectorFactory(const char* connectorName) - : ConnectorFactory(connectorName) {} - - std::shared_ptr newConnector( - const std::string& id, - std::shared_ptr config, - folly::Executor* ioExecutor = nullptr, - folly::Executor* cpuExecutor = nullptr) override { - return std::make_shared(id, config, ioExecutor); - } -}; - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConnectorSplit.cpp b/velox/connectors/kafka/KafkaConnectorSplit.cpp deleted file mode 100644 index 14a5d3b8adc3..000000000000 --- a/velox/connectors/kafka/KafkaConnectorSplit.cpp +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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 "velox/connectors/kafka/KafkaConnectorSplit.h" -#include -#include - -namespace facebook::velox::connector::kafka { - -std::string KafkaConnectorSplit::topicPartitonsToString( - const cppkafka::TopicPartitionList& tps) { - if (tps.empty()) { - return ""; - } - std::stringstream s; - for (size_t i = 0; i < tps.size() - 1; ++i) { - s << "[" << tps[i].get_topic() << "," << tps[i].get_partition() << "]"; - s << ","; - } - s << "[" << tps[tps.size() - 1].get_topic() << "," - << tps[tps.size() - 1].get_partition() << "]"; - return s.str(); -} - -std::string KafkaConnectorSplit::toString() const { - return fmt::format( - "Kafka connector split, connectorId: {}, bootstrap servers:{}, topic partitions:{}, group id:{}", - clientId_, - bootstrapServers_, - KafkaConnectorSplit::topicPartitonsToString(getCppKafkaTopicPartitions()), - groupId_); -} - -folly::dynamic KafkaConnectorSplit::serialize() const { - folly::dynamic obj = folly::dynamic::object; - obj["connectorId"] = clientId_; - obj["bootstrapServers"] = bootstrapServers_; - obj["groupId"] = groupId_; - obj["format"] = format_; - obj["enableAutoCommit"] = enableAutoCommit_; - obj["autoResetOffset"] = autoResetOffset_; - folly::dynamic tpObjs = folly::dynamic::array; - size_t j = 0; - for (const auto& tp : topicPartitions_) { - std::string topic = tp.first; - for (size_t i = 0; i < tp.second.size(); ++i) { - folly::dynamic d = folly::dynamic::object; - d["topic"] = topic; - d["partition"] = tp.second[i].first; - d["offset"] = tp.second[i].second; - tpObjs[j] = d; - j++; - } - } - obj["topicPartitions"] = tpObjs; - return obj; -} - -std::shared_ptr KafkaConnectorSplit::create( - const folly::dynamic& obj) { - std::unordered_map>> topics; - if (obj["topicPartitions"].isArray()) { - const auto tpObjs = obj["topicPartitions"]; - for (const auto& tp : tpObjs) { - if (tp.isObject()) { - const auto topic = tp["topic"]; - const auto partition = tp["partition"]; - const auto offset = tp["offset"]; - const auto it = topics.find(topic.asString()); - std::pair p( - static_cast(partition.asInt()), - static_cast(offset.asInt())); - if (it != topics.end()) { - std::vector>& partitionOffset = - it->second; - partitionOffset.emplace_back(p); - } else { - std::vector> partitionOffset; - partitionOffset.emplace_back(p); - topics[topic.asString()] = partitionOffset; - } - } - } - } - return std::make_shared( - obj["connectorId"].asString(), - obj["bootstrapServers"].asString(), - obj["groupId"].asString(), - obj["format"].asString(), - obj["enableAutoCommit"].asBool(), - obj["autoResetOffset"].asString(), - topics); -} - -void KafkaConnectorSplit::registerSerDe() { - auto& registry = DeserializationRegistryForSharedPtr(); - registry.Register("KafkaConnectorSplit", KafkaConnectorSplit::create); -} - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConnectorSplit.h b/velox/connectors/kafka/KafkaConnectorSplit.h deleted file mode 100644 index 9bf570ec0977..000000000000 --- a/velox/connectors/kafka/KafkaConnectorSplit.h +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ - -#pragma once - -#include "velox/connectors/Connector.h" -#include -#include -#include - -namespace facebook::velox::connector::kafka { - -/// The split information for kafka connector. -struct KafkaConnectorSplit : public ConnectorSplit { - std::string bootstrapServers_; - std::string groupId_; - std::string clientId_; - std::string format_; - bool enableAutoCommit_; - std::string autoResetOffset_; - std::unordered_map>> - topicPartitions_; - - explicit KafkaConnectorSplit( - const std::string& connectorId, - const std::string& bootstrapServers, - const std::string& groupId, - const std::string& format, - const bool enableAutoCommit, - const std::string& autoResetOffset, - std::unordered_map>>& - tps) - : ConnectorSplit(connectorId), - bootstrapServers_(bootstrapServers), - groupId_(groupId), - clientId_(connectorId), - format_(format), - enableAutoCommit_(enableAutoCommit), - autoResetOffset_(autoResetOffset), - topicPartitions_(tps) {} - - cppkafka::TopicPartitionList getCppKafkaTopicPartitions() const { - cppkafka::TopicPartitionList topicPartitions; - for (const auto& p : topicPartitions_) { - std::string topic = p.first; - for (const auto& partition : p.second) { - cppkafka::TopicPartition topicPartition( - topic, static_cast(partition.first)); - if (partition.second >= 0) { - topicPartition.set_offset(partition.second); - } - topicPartitions.emplace_back(topicPartition); - } - } - return topicPartitions; - } - - static std::string topicPartitonsToString(const cppkafka::TopicPartitionList& tps); - - std::string toString() const override; - - folly::dynamic serialize() const override; - - static std::shared_ptr create(const folly::dynamic& obj); - - static void registerSerDe(); -}; -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConsumer.cpp b/velox/connectors/kafka/KafkaConsumer.cpp deleted file mode 100644 index 54035213f93f..000000000000 --- a/velox/connectors/kafka/KafkaConsumer.cpp +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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 "velox/connectors/kafka/KafkaConsumer.h" -#include "velox/connectors/kafka/KafkaConnectorSplit.h" -#include -#include -#include - -namespace facebook::velox::connector::kafka { - -void KafkaConsumer::subscribe(const std::vector& topics) { - auto topicsToString = [&]() -> std::string { - if (topics.empty()) { - return ""; - } - std::stringstream s; - for (size_t i = 0; i < topics.size() - 1; ++i) { - s << topics[i]; - s << ","; - } - s << topics[topics.size() - 1]; - return s.str(); - }; - VELOX_CHECK_NOT_NULL( - consumer_.get(), - "Failed to subscribe to topics: {}, as the cppkafka consumer is null.", - topicsToString()); - consumer_->subscribe(topics); -} - -const cppkafka::TopicPartitionList KafkaConsumer::getTopicPartitions( - const std::string& topic, - const std::string& startupMode) { - cppkafka::TopicPartitionList tps; - auto metadata = consumer_->get_metadata(); - const auto& topics = metadata.get_topics(); - for (const cppkafka::TopicMetadata& topicMetadata : topics) { - if (topicMetadata.get_name() == topic) { - const auto& partitions = topicMetadata.get_partitions(); - for (const auto& partition : partitions) { - cppkafka::TopicPartition topicPartition( - topic, static_cast(partition.get_id())); - auto offsets = consumer_->query_offsets(topicPartition); - if (startupMode == "earliest-offsets") { - topicPartition.set_offset(std::get<0>(offsets)); - } else if (startupMode == "latest-offsets") { - topicPartition.set_offset(std::get<1>(offsets)); - } - tps.emplace_back(topicPartition); - } - } - } - VELOX_CHECK_GT(tps.size(), 0, "Failed to get partitions of topic:{}", topic); - return tps; -} - -const void KafkaConsumer::setTopicPartitionsOffset(cppkafka::TopicPartitionList& tps, const std::string& startupMode) { - for (cppkafka::TopicPartition & tp : tps) { - auto offsets = consumer_->query_offsets(tp); - if (startupMode == "earliest-offsets") { - tp.set_offset(std::get<0>(offsets)); - } else if (startupMode == "latest-offsets") { - tp.set_offset(std::get<1>(offsets)); - } - } -} - -void KafkaConsumer::assign(const cppkafka::TopicPartitionList& tps) { - std::string tpsString = KafkaConnectorSplit::topicPartitonsToString(tps); - VELOX_CHECK_NOT_NULL( - consumer_.get(), - "Failed to assign topic partitions: {}, as the cppkafka consumer is null.", - tpsString); - consumer_->assign(tps); -} - -const void KafkaConsumer::consumeBatch( - std::vector& res, - size_t& msgBytes) { - const std::vector msgs = - consumer_->poll_batch(pollBatchSize_); - for (const auto& msg : msgs) { - const std::string& msgData = msg.get_payload(); - msgBytes += msgData.size(); - res.emplace_back(msgData); - } -} - -const std::vector KafkaConsumer::getSubscribedTopics() { - return consumer_->get_subscription(); -} - -const cppkafka::TopicPartitionList KafkaConsumer::getAssignedTopicPartitions() { - return consumer_->get_assignment(); -} - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaConsumer.h b/velox/connectors/kafka/KafkaConsumer.h deleted file mode 100644 index b27a0d76c79f..000000000000 --- a/velox/connectors/kafka/KafkaConsumer.h +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ - -#pragma once - -#include "velox/connectors/kafka/KafkaConfig.h" -#include -#include - -namespace facebook::velox::connector::kafka { - -using CppKafkaConsumerPtr = std::shared_ptr; - -/// Class for consume records from kafka topic -class KafkaConsumer { - public: - KafkaConsumer( - const CppKafkaConsumerPtr& consumer, - const uint32_t pollTimeOut, - const uint32_t pollBatchSize) - : consumer_(consumer), - pollTimeOutMillis_(pollTimeOut), - pollBatchSize_(pollBatchSize > 1 ? pollBatchSize : ConnectionConfig::defaultPollBatchSize) {} - - ~KafkaConsumer() {} - - /// Get the parititions of the given `topic`, and get the offset accroding to - /// the given `startupMode`. - const cppkafka::TopicPartitionList getTopicPartitions( - const std::string& topic, - const std::string& startupMode); - /// Set topic partitions offset according to the given `startupMode`. - const void setTopicPartitionsOffset( - cppkafka::TopicPartitionList& tps, - const std::string& startupMode); - /// Subscribe to the kafka topics. - void subscribe(const std::vector& topics); - /// Assign the consumer to the given topic partitions. - void assign(const cppkafka::TopicPartitionList& tps); - /// Consume a batch of messages. - const void consumeBatch(std::vector& msgs, size_t& msgBytes); - /// For test, Get the kafka topics that already subscribed. - const std::vector getSubscribedTopics(); - /// For test, Get the kafka assigned topic and partitions. - const cppkafka::TopicPartitionList getAssignedTopicPartitions(); - - private: - /// The kafka consume handle by using `cppkafka`. - CppKafkaConsumerPtr consumer_; - /// The timeout milliseconds for consuming kafka. - std::chrono::milliseconds pollTimeOutMillis_; - /// The batch size for consuming kafka. - uint32_t pollBatchSize_; -}; - -using KafkaConsumerPtr = std::shared_ptr; -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaDataSource.cpp b/velox/connectors/kafka/KafkaDataSource.cpp deleted file mode 100644 index 70ee1beb03b6..000000000000 --- a/velox/connectors/kafka/KafkaDataSource.cpp +++ /dev/null @@ -1,165 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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 "velox/connectors/kafka/KafkaDataSource.h" -#include "velox/common/base/RuntimeMetrics.h" -#include "velox/connectors/kafka/KafkaTableHandle.h" -#include "velox/connectors/kafka/format/CSVRecordDeserializer.h" -#include "velox/connectors/kafka/format/RawRecordDeserializer.h" -#include "velox/connectors/kafka/format/StreamJSONRecordDeserializer.h" -#include "velox/connectors/kafka/KafkaConnectorSplit.h" -#include "velox/vector/BaseVector.h" - -namespace facebook::velox::connector::kafka { - -KafkaDataSource::KafkaDataSource( - const RowTypePtr& outputType, - const TableHandlePtr& tableHandle, - const ConnectorQueryCtx* connectorQueryCtx, - const ConnectionConfigPtr& config) - : queryCtx_(connectorQueryCtx), - config_(config), - outputType_(outputType), - batchSize_(config_->getDataBatchSize()) { - VELOX_CHECK(batchSize_ > 0, "Batch size config value must greater than 0."); - const std::shared_ptr kafkaTableHandle = - std::dynamic_pointer_cast(tableHandle); - if (kafkaTableHandle) { - const std::unordered_map& tableParams = - kafkaTableHandle->tableParameters(); - config_ = config_->setConfigs(tableParams); - } else { - VELOX_FAIL( - "The table handle {} is not supported for kafka data source.", - tableHandle->connectorId()); - } - if (consumerCanbeCreated()) { - cppkafka::Configuration cppKafkaConfig = - config_->getCppKafkaConfiguration(); - createConsumer(cppKafkaConfig); - } - createCachedQueue(batchSize_); - createRecordDeserializer(config_->getFormat(), outputType_); -} - -bool KafkaDataSource::consumerCanbeCreated() { - return config_->exists(ConnectionConfig::kBootstrapServers) && - config_->exists(ConnectionConfig::kClientId) && - config_->exists(ConnectionConfig::kTopic) && - config_->exists(ConnectionConfig::kGroupId) && - config_->exists(ConnectionConfig::kFormat) && !consumer_.get(); -} - -void KafkaDataSource::createConsumer(cppkafka::Configuration& config) { - VELOX_CHECK_NULL( - consumer_.get(), - "Failed to create kafka consumer as the consumer is not null"); - CppKafkaConsumerPtr cppKafkaConsumer = - std::make_shared(config); - cppKafkaConsumer->set_destroy_flags(RD_KAFKA_DESTROY_F_NO_CONSUMER_CLOSE); - consumer_ = std::make_shared( - cppKafkaConsumer, config_->getPollTimeoutMills(), batchSize_); - std::string topic = config_->getTopic(); - topics_.emplace_back(topic); - consumer_->subscribe(topics_); -} - -void KafkaDataSource::createCachedQueue(const uint32_t size) { - VELOX_CHECK_GT( - size, 0, "Kafka consume message queue size must greater than 0"); - queue_.reserve(size); -} - -void KafkaDataSource::createRecordDeserializer( - const std::string& format, - const RowTypePtr& outputType) { - if (format == "json") { - deserializer_ = std::make_shared( - outputType, queryCtx_->memoryPool()); - } else if (format == "csv") { - deserializer_ = std::make_shared( - outputType, queryCtx_->memoryPool()); - } else if (format == "raw") { - deserializer_ = std::make_shared( - outputType, queryCtx_->memoryPool()); - } else { - VELOX_FAIL_UNSUPPORTED_INPUT_UNCATCHABLE( - "The data format {} is not supported for kafka.", format); - } - outRow_ = RowVector::createEmpty(outputType_, queryCtx_->memoryPool()); - outRow_->resize(1); -} - -void KafkaDataSource::addSplit(ConnectorSplitPtr split) { - KafkaConnectorSplit* kafkaConnectorSplit = - static_cast(split.get()); - VELOX_CHECK_NOT_NULL( - kafkaConnectorSplit, - "Failed to add split, because the kafka connector split is null."); - VELOX_CHECK_NOT_NULL( - consumer_.get(), - "Failed to add split, because the kafka consumer is null."); - cppkafka::TopicPartitionList topicPartitions = - kafkaConnectorSplit->getCppKafkaTopicPartitions(); - if (topicPartitions.size() == 0) { - const auto tps = - consumer_->getTopicPartitions(topics_[0], config_->getStartupMode()); - consumer_->assign(tps); - } else { - consumer_->setTopicPartitionsOffset(topicPartitions, config_->getStartupMode()); - consumer_->assign(topicPartitions); - } -} - -std::optional KafkaDataSource::next( - uint64_t, - velox::ContinueFuture&) { - std::optional res; - size_t consumedMsgBytes = 0; - if (queue_.empty()) { - // consume the data batch from kafka, and stored the consumed data in the queue. - consumer_->consumeBatch(queue_, consumedMsgBytes); - consumePos_ = 0; - // If nothing consumed, return directly. - if (consumedMsgBytes == 0) { - return res; - } - } - outRow_->prepareForReuse(); - // If batchSize > 1 and set `processDataSize = queue.size`, means to process the entrie batch that consumed and stored in the `queue` at once; - // If batchSize = 1 and set `processDataSize = 1`, means to process the consumed batch data one by one. - size_t processDataSize = batchSize_ > 1 ? queue_.size() : batchSize_; - outRow_->resize(processDataSize); - // Deserialize the consumed data. The `processDataSize` determines how many data would be deserialized at once. - for (size_t pos = 0; pos < processDataSize; ++pos) { - deserializer_->deserialize(queue_[pos + consumePos_], pos, outRow_); - completedBytes_ += queue_[pos + consumePos_].size(); - completedRows_ += 1; - } - res.emplace(std::dynamic_pointer_cast(outRow_)); - consumePos_ += processDataSize; - if (consumePos_ >= queue_.size()) { - queue_.clear(); - consumePos_ = 0; - } - return res; -} - -std::unordered_map KafkaDataSource::runtimeStats() { - std::unordered_map stats; - return stats; -} -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaDataSource.h b/velox/connectors/kafka/KafkaDataSource.h deleted file mode 100644 index 02c6b7e76709..000000000000 --- a/velox/connectors/kafka/KafkaDataSource.h +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ -#pragma once - -#include "velox/common/base/RuntimeMetrics.h" -#include "velox/common/future/VeloxPromise.h" -#include "velox/connectors/Connector.h" -#include "velox/connectors/kafka/KafkaConfig.h" -#include "velox/connectors/kafka/KafkaConsumer.h" -#include "velox/type/Filter.h" -#include "velox/type/Type.h" -#include - -namespace facebook::velox::connector::kafka { - -using TableHandlePtr = std::shared_ptr; -using ConnectorSplitPtr = std::shared_ptr; - -class KafkaRecordDeserializer; -using KafkaRecordDeserializerPtr = std::shared_ptr; - -class KafkaDataSource : public DataSource { - public: - KafkaDataSource( - const RowTypePtr& outputType, - const TableHandlePtr& tableHandle, - const ConnectorQueryCtx* connectorQueryCtx, - const ConnectionConfigPtr& connectionConfig); - - /// Create a kafka connection to the given topics and partitions. - void addSplit(ConnectorSplitPtr split) override; - - /// Fetch record from the consumed records. - std::optional next(uint64_t size, velox::ContinueFuture& future) - override; - - void addDynamicFilter( - column_index_t outputChannel, - const std::shared_ptr& filter) override {} - - uint64_t getCompletedBytes() override { - return completedBytes_; - } - - uint64_t getCompletedRows() override { - return completedRows_; - } - - std::unordered_map runtimeStats() override; - - /// For test. - const KafkaConsumerPtr & getConsumer() const { - return consumer_; - } - - /// For test. - const KafkaRecordDeserializerPtr & getDeserializer() const { - return deserializer_; - } - - private: - /// The context of kafka connector for query. - const ConnectorQueryCtx* queryCtx_; - /// The connection config for kafka. - ConnectionConfigPtr config_; - /// The type of output. - RowTypePtr outputType_; - /// The kafka topics to be consumed. - std::vector topics_; - /// The kafka consumer. - KafkaConsumerPtr consumer_; - /// The kafka record deserializer. - KafkaRecordDeserializerPtr deserializer_; - /// Count how many rows consumed. - uint64_t completedRows_ = 0; - /// Count how many bytes consumed. - uint64_t completedBytes_ = 0; - /// The output row to be returned. - VectorPtr outRow_; - /// The batch size of data are consumed/processed at once . - /// If batchSize_ = 1, it means to process the consumed messages one by one, which likes valina flink does. - /// If batchSize_ > 1, it means to process the consumed messages by a entrie batch. - uint64_t batchSize_; - /// The cache queue for storing consumed data. - std::vector queue_; - /// The consumed position of the cache queue. - size_t consumePos_ = 0; - - /// Whether consumer can be created. - bool consumerCanbeCreated(); - - /// Create kafka consumer from the configuration. - void createConsumer(cppkafka::Configuration& config); - - /// Create message queue with given size. - void createCachedQueue(const uint32_t size); - - /// Create deserializer to deserialize the consumed recored to the given row - /// type. - void createRecordDeserializer( - const std::string& format, - const RowTypePtr& outputType); -}; - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaTableHandle.cpp b/velox/connectors/kafka/KafkaTableHandle.cpp deleted file mode 100644 index 896a564dc132..000000000000 --- a/velox/connectors/kafka/KafkaTableHandle.cpp +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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 "velox/connectors/kafka/KafkaTableHandle.h" - -namespace facebook::velox::connector::kafka { - -std::string KafkaTableHandle::toString() const { - std::stringstream out; - out << "table: " << tableName_; - if (dataColumns_) { - out << ", data columns: " << dataColumns_->toString(); - } - if (!tableParameters_.empty()) { - std::map orderedTableParameters{ - tableParameters_.begin(), tableParameters_.end()}; - out << ", table parameters: ["; - bool firstParam = true; - for (const auto& param : orderedTableParameters) { - if (!firstParam) { - out << ", "; - } - out << param.first << ":" << param.second; - firstParam = false; - } - out << "]"; - } - return out.str(); -} - -folly::dynamic KafkaTableHandle::serialize() const { - folly::dynamic obj = ConnectorTableHandle::serializeBase("KafkaTableHandle"); - obj["tableName"] = tableName_; - if (dataColumns_) { - obj["dataColumns"] = dataColumns_->serialize(); - } - folly::dynamic tableParameters = folly::dynamic::object; - for (const auto& param : tableParameters_) { - tableParameters[param.first] = param.second; - } - obj["tableParameters"] = tableParameters; - return obj; -} - -ConnectorTableHandlePtr KafkaTableHandle::create( - const folly::dynamic& obj, - void* context) { - auto connectorId = obj["connectorId"].asString(); - auto tableName = obj["tableName"].asString(); - RowTypePtr dataColumns; - if (auto it = obj.find("dataColumns"); it != obj.items().end()) { - dataColumns = ISerializable::deserialize(it->second, context); - } - - std::unordered_map tableParameters{}; - const auto& tableParametersObj = obj["tableParameters"]; - for (const auto& key : tableParametersObj.keys()) { - const auto& value = tableParametersObj[key]; - tableParameters.emplace(key.asString(), value.asString()); - } - - return std::make_shared( - connectorId, - tableName, - dataColumns, - tableParameters); -} - -void KafkaTableHandle::registerSerDe() { - auto& registry = DeserializationWithContextRegistryForSharedPtr(); - registry.Register("KafkaTableHandle", create); -} -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/KafkaTableHandle.h b/velox/connectors/kafka/KafkaTableHandle.h deleted file mode 100644 index 9cab391987aa..000000000000 --- a/velox/connectors/kafka/KafkaTableHandle.h +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ - -#pragma once - -#include "velox/connectors/Connector.h" -#include "velox/core/ITypedExpr.h" -#include "velox/type/Filter.h" -#include "velox/type/Type.h" -#include - -namespace facebook::velox::connector::kafka { - -/// The table handle for kafka connector. -class KafkaTableHandle : public ConnectorTableHandle { - public: - KafkaTableHandle( - std::string connectorId, - std::string tableName, - const RowTypePtr& dataColumns = nullptr, - const std::unordered_map& tableParameters = {}) - : ConnectorTableHandle(connectorId), - tableName_(tableName), - dataColumns_(dataColumns), - tableParameters_(tableParameters) {} - - const std::string& tableName() { - return tableName_; - } - - const RowTypePtr& dataColumns() { - return dataColumns_; - } - - const std::unordered_map& tableParameters() { - return tableParameters_; - } - - std::unordered_map& getTableParameters() { - return tableParameters_; - } - - const std::string& name() const override { - return tableName_; - } - - std::string toString() const override; - - folly::dynamic serialize() const override; - - static ConnectorTableHandlePtr create( - const folly::dynamic& obj, - void* context); - - static void registerSerDe(); - - private: - std::string tableName_; - RowTypePtr dataColumns_; - std::unordered_map tableParameters_; -}; - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/CMakeLists.txt b/velox/connectors/kafka/format/CMakeLists.txt deleted file mode 100644 index 12f9cc0a274c..000000000000 --- a/velox/connectors/kafka/format/CMakeLists.txt +++ /dev/null @@ -1,21 +0,0 @@ -# Copyright (c) Facebook, Inc. and its affiliates. -# -# Licensed 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. -velox_add_library(velox_kafka_connector_format OBJECT - JSONRecordDeserializer.cpp StreamJSONRecordDeserializer.cpp) - -velox_link_libraries( - velox_kafka_connector_format - velox_common_io - velox_connector - simdjson::simdjson) diff --git a/velox/connectors/kafka/format/CSVRecordDeserializer.h b/velox/connectors/kafka/format/CSVRecordDeserializer.h deleted file mode 100644 index ae1f389baa94..000000000000 --- a/velox/connectors/kafka/format/CSVRecordDeserializer.h +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ -#pragma once - -#include "velox/connectors/kafka/format/KafkaRecordDeserializer.h" - -namespace facebook::velox::connector::kafka { -/// Class for kafka record deserialization of csv format. -class KafkaCSVRecordDeserializer : public KafkaRecordDeserializer { - public: - KafkaCSVRecordDeserializer( - const RowTypePtr& outputType, - memory::MemoryPool* memoryPool) - : KafkaRecordDeserializer(outputType, memoryPool) {} - - const void deserialize( - const std::string & message, - const size_t index, - VectorPtr& vec) override { - VELOX_NYI("Not implemented."); - } -}; - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/JSONRecordDeserializer.cpp b/velox/connectors/kafka/format/JSONRecordDeserializer.cpp deleted file mode 100644 index 2d82979a0bd0..000000000000 --- a/velox/connectors/kafka/format/JSONRecordDeserializer.cpp +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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 "velox/connectors/kafka/format/JSONRecordDeserializer.h" - -namespace facebook::velox::connector::kafka { - -using IntDeserializer = BaseDeserializer; -using BigIntDeseralizer = BaseDeserializer; -using SmallIntDeserializer = BaseDeserializer; -using TinyIntDeserializer = BaseDeserializer; -using BoolDeserializer = BaseDeserializer; -using FloatDeserializer = BaseDeserializer; -using DoubleDeserializer = BaseDeserializer; -using StringDeserializer = BaseDeserializer; -using TimestampDeserializer = BaseDeserializer; - -const std::shared_ptr JSONDeserializer::create( - const TypePtr& type) { - const TypeKind kind = type->kind(); - if (kind == TypeKind::INTEGER) { - return std::make_shared(); - } else if (kind == TypeKind::BIGINT) { - return std::make_shared(); - } else if (kind == TypeKind::SMALLINT) { - return std::make_shared(); - } else if (kind == TypeKind::TINYINT) { - return std::make_shared(); - } else if (kind == TypeKind::BOOLEAN) { - return std::make_shared(); - } else if (kind == TypeKind::VARCHAR) { - return std::make_shared(); - } else if (kind == TypeKind::REAL) { - return std::make_shared(); - } else if (kind == TypeKind::DOUBLE) { - return std::make_shared(); - } else if (kind == TypeKind::TIMESTAMP) { - return std::make_shared(); - } else if (kind == TypeKind::ROW) { - const RowTypePtr rowType = std::dynamic_pointer_cast(type); - const std::vector fieldNames = rowType->names(); - const std::vector fieldTypes = rowType->children(); - std::vector> deserializers; - for (const auto& fieldType : fieldTypes) { - deserializers.emplace_back(create(fieldType)); - } - return std::make_shared( - fieldNames, fieldTypes, deserializers); - } else if (kind == TypeKind::ARRAY) { - const std::shared_ptr arrayType = - std::dynamic_pointer_cast(type); - const TypePtr& elementType = arrayType->elementType(); - const std::shared_ptr elementDeserializer = - create(elementType); - return std::make_shared( - elementType, elementDeserializer); - } else if (kind == TypeKind::MAP) { - const std::shared_ptr mapType = - std::dynamic_pointer_cast(type); - const TypePtr& keyType = mapType->keyType(); - const TypePtr& valueType = mapType->valueType(); - const std::shared_ptr keyDeserializer = create(keyType); - const std::shared_ptr valueDeserializer = - create(valueType); - return std::make_shared( - keyType, valueType, keyDeserializer, valueDeserializer); - } else { - VELOX_FAIL("The type is not supported: {}", type); - } -} - -const void KafkaJSONRecordDeserializer::deserialize( - const std::string& message, - const size_t index, - VectorPtr& vec) { - try { - Element element; - parser_->parse(message.data(), message.size()).get(element); - if (element.is_null()) { - vec->setNull(true, index); - } else { - deserializer_->deserialize(std::move(element), index, vec); - } - } catch (const std::exception& e) { - LOG(WARNING) << "Failed to deserialize record: " << message - << " , error: " << e.what(); - } -} - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/JSONRecordDeserializer.h b/velox/connectors/kafka/format/JSONRecordDeserializer.h deleted file mode 100644 index 9f518d962e7c..000000000000 --- a/velox/connectors/kafka/format/JSONRecordDeserializer.h +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ - -#pragma once - -#include "velox/connectors/kafka/format/KafkaRecordDeserializer.h" -#include "velox/type/StringView.h" -#include "velox/type/Timestamp.h" -#include "velox/type/TimestampConversion.h" -#include "velox/type/Type.h" -#include "simdjson.h" -#include -#include - -namespace facebook::velox::connector::kafka { - -using Element = simdjson::dom::element; -using Elements = std::vector; - -struct JSONDeserializer { - public: - virtual inline const void - deserialize(const Element& e, const size_t index, VectorPtr& vec) = 0; - - static const std::shared_ptr create(const TypePtr& type); -}; - -template -struct BaseDeserializer : public JSONDeserializer { - public: - inline const void - deserialize(const Element& e, const size_t index, VectorPtr& vec) override { - auto flat = std::dynamic_pointer_cast>(vec); - if constexpr (std::is_same_v || std::is_same_v) { - flat->set(index, static_cast(e.get_double().value())); - } else if constexpr (std::is_same_v) { - flat->set(index, e.get_bool().value()); - } else if constexpr (std::is_same_v) { - const auto s = e.get_string().value(); - flat->set(index, StringView(s.data(), s.size())); - } else if constexpr (std::is_same_v) { - const auto s = e.get_string().value(); - const auto timestamp = - util::fromTimestampString( - s.data(), s.size(), util::TimestampParseMode::kLegacyCast) - .thenOrThrow(folly::identity, [&](const Status& status) { - VELOX_FAIL("error while parse timestamp: {}", status.message()); - }); - flat->set(index, timestamp); - } else if constexpr ( - std::is_same_v || std::is_same_v || - std::is_same_v || std::is_same_v) { - flat->set(index, static_cast(e.get_int64().value())); - } else { - VELOX_FAIL("The type {} is not supported", typeid(T).name()); - } - } -}; - -struct RowDeserializer : public JSONDeserializer { - public: - RowDeserializer( - const std::vector& fieldNames, - const std::vector& fieldTypes, - const std::vector>& deserializers) - : fieldNames_(fieldNames), - fieldTypes_(fieldTypes), - deserializers_(deserializers) {} - - inline const void - deserialize(const Element& e, const size_t index, VectorPtr& vec) override { - RowVectorPtr rowVector = std::dynamic_pointer_cast(vec); - std::vector& rowFields = rowVector->children(); - for (size_t i = 0; i < fieldNames_.size(); ++i) { - if (e[fieldNames_[i]].is_null()) { - rowFields[i]->setNull(index, true); - } else { - deserializers_[i]->deserialize(e[fieldNames_[i]], index, rowFields[i]); - } - } - } - - private: - std::vector fieldNames_; - std::vector fieldTypes_; - std::vector> deserializers_; -}; - -struct ArrayDeserializer : public JSONDeserializer { - public: - ArrayDeserializer( - const TypePtr& elementType, - const std::shared_ptr& deserializer) - : elementType_(elementType), elementDeserializer_(deserializer) {} - - inline const void - deserialize(const Element& e, const size_t index, VectorPtr& vec) override {} - - private: - TypePtr elementType_; - std::shared_ptr elementDeserializer_; -}; - -struct MapDeserializer : public JSONDeserializer { - public: - MapDeserializer( - const TypePtr& keyType, - const TypePtr& valueType, - const std::shared_ptr& keyDeserializer, - const std::shared_ptr& valueDeserializer) - : keyType_(keyType), - valueType_(valueType), - keyDeserializer_(keyDeserializer), - valueDeserializer_(valueDeserializer) {} - - inline const void - deserialize(const Element& e, const size_t index, VectorPtr& vec) override {} - - private: - TypePtr keyType_; - TypePtr valueType_; - std::shared_ptr keyDeserializer_; - std::shared_ptr valueDeserializer_; -}; - -/// Class for kafka record deserialization of json format. -class KafkaJSONRecordDeserializer : public KafkaRecordDeserializer { - public: - KafkaJSONRecordDeserializer( - const RowTypePtr& outputType, - memory::MemoryPool* memoryPool) - : KafkaRecordDeserializer(outputType, memoryPool), - deserializer_(std::dynamic_pointer_cast( - JSONDeserializer::create(outputType))), - parser_(std::make_shared()) {} - - const void deserialize( - const std::string& message, - const size_t index, - VectorPtr& vec) override; - - private: - std::shared_ptr deserializer_; - std::shared_ptr parser_; -}; - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/KafkaRecordDeserializer.h b/velox/connectors/kafka/format/KafkaRecordDeserializer.h deleted file mode 100644 index b86b91dace61..000000000000 --- a/velox/connectors/kafka/format/KafkaRecordDeserializer.h +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ -#pragma once - -#include "velox/common/memory/MemoryPool.h" -#include "velox/type/Type.h" -#include "velox/vector/ComplexVector.h" -#include "velox/vector/FlatVector.h" - -namespace facebook::velox::connector::kafka { - -/// Base class for kafka record deserialization. -class KafkaRecordDeserializer { - public: - KafkaRecordDeserializer( - const RowTypePtr& outputType, - memory::MemoryPool* memoryPool) - : outputType_(outputType), memoryPool_(memoryPool) { - VELOX_CHECK_GT( - outputType_->size(), - 0, - "Output type size of record deserializer must great than 0."); - VELOX_CHECK_NOT_NULL( - memoryPool_, "Memory pool of record deserializer must not be null."); - } - - virtual ~KafkaRecordDeserializer() = default; - - /// Deserialize a single message. - virtual const void deserialize( - const std::string & message, - const size_t index, - VectorPtr& vec) = 0; - - protected: - RowTypePtr outputType_; - memory::MemoryPool* memoryPool_; -}; - -using KafkaRecordDeserializerPtr = std::shared_ptr; - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/RawRecordDeserializer.h b/velox/connectors/kafka/format/RawRecordDeserializer.h deleted file mode 100644 index 57371b2827ea..000000000000 --- a/velox/connectors/kafka/format/RawRecordDeserializer.h +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ -#pragma once - -#include "velox/connectors/kafka/format/KafkaRecordDeserializer.h" -#include "velox/type/Type.h" - -namespace facebook::velox::connector::kafka { -/// Class for kafka record deserialization of raw format. -class KafkaRawRecordDeserializer : public KafkaRecordDeserializer { - public: - KafkaRawRecordDeserializer( - const RowTypePtr& outputType, - memory::MemoryPool* memoryPool) - : KafkaRecordDeserializer(outputType, memoryPool) { - VELOX_CHECK_EQ( - outputType_->size(), - 1, - "Output type size of raw deserializer must be 1."); - const TypePtr& childType = outputType_->childAt(0); - VELOX_CHECK_EQ( - childType->kind(), - TypeKind::VARCHAR, - "Output type must be Row(String)."); - } - - const void deserialize( - const std::string& message, - const size_t index, - VectorPtr& vec) override { - RowVectorPtr rowVector = std::dynamic_pointer_cast(vec); - VELOX_CHECK_EQ( - rowVector->childrenSize(), - 1, - "The raw record vector children size {} is not 1", - rowVector->childrenSize()); - VectorPtr& childVector = rowVector->children()[0]; - auto flat = std::dynamic_pointer_cast>(childVector); - flat->set(index, StringView(message.data(), message.size())); - } -}; - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/StreamJSONRecordDeserializer.cpp b/velox/connectors/kafka/format/StreamJSONRecordDeserializer.cpp deleted file mode 100644 index fdb29a180e22..000000000000 --- a/velox/connectors/kafka/format/StreamJSONRecordDeserializer.cpp +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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 "velox/connectors/kafka/format/StreamJSONRecordDeserializer.h" - -namespace facebook::velox::connector::kafka { - -const std::shared_ptr StreamJSONDeserializer::create( - const TypePtr& type) { - const TypeKind kind = type->kind(); - if (kind == TypeKind::INTEGER) { - return std::make_shared(); - } else if (kind == TypeKind::BIGINT) { - return std::make_shared(); - } else if (kind == TypeKind::HUGEINT) { - return std::make_shared(); - } else if (kind == TypeKind::VARCHAR) { - return std::make_shared(); - } else if (kind == TypeKind::TIMESTAMP) { - return std::make_shared(); - } else if (kind == TypeKind::ROW) { - const RowTypePtr rowType = std::dynamic_pointer_cast(type); - const std::vector fieldNames = rowType->names(); - const std::vector fieldTypes = rowType->children(); - std::vector> deserializers; - for (const auto& fieldType : fieldTypes) { - deserializers.emplace_back(create(fieldType)); - } - return std::make_shared( - fieldNames, fieldTypes, deserializers); - } else { - VELOX_FAIL("The type is not supported: {}", type); - } -} - -const void KafkaStreamJSONRecordDeserializer::deserialize( - const std::string& message, - const size_t index, - VectorPtr& vec) { - try { - simdjson::padded_string_view json_padded( - message.data(), - message.size(), - message.size() + simdjson::SIMDJSON_PADDING); - simdjson::ondemand::document doc = parser_->iterate(json_padded); - JSONValue value = doc.get_value(); - if (value.is_null()) { - vec->setNull(index, true); - } else { - deserializer_->deserialize(value, index, vec); - } - } catch (const std::exception& e) { - LOG(WARNING) << "Failed to deserialize record: " << message - << " , error: " << e.what(); - } -} - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/format/StreamJSONRecordDeserializer.h b/velox/connectors/kafka/format/StreamJSONRecordDeserializer.h deleted file mode 100644 index 0f1b7971868c..000000000000 --- a/velox/connectors/kafka/format/StreamJSONRecordDeserializer.h +++ /dev/null @@ -1,165 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ -#pragma once - -#include "velox/connectors/kafka/format/StreamJSONRecordDeserializer.h" -#include "velox/connectors/kafka/format/KafkaRecordDeserializer.h" - -#include "velox/type/StringView.h" -#include "velox/type/Timestamp.h" -#include "velox/type/Type.h" -#include "simdjson.h" -#include -#include - -namespace facebook::velox::connector::kafka { - -using JSONDoc = simdjson::ondemand::document; -using JSONValue = simdjson::ondemand::value; -using JSONRow = simdjson::ondemand::object; -using JSONArray = simdjson::ondemand::array; - -struct StreamJSONDeserializer { - public: - virtual ~StreamJSONDeserializer() = default; - - static const std::shared_ptr create( - const TypePtr& type); - - virtual const void - deserialize(JSONValue& e, const size_t index, VectorPtr& vec) = 0; -}; - -struct StreamIntDeserializer : public StreamJSONDeserializer { - public: - StreamIntDeserializer() {} - - inline const void - deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { - auto flat = std::dynamic_pointer_cast>(vec); - flat->set(index, static_cast(e.get_int64())); - } -}; - -struct StreamBigIntDeserializer : public StreamJSONDeserializer { - public: - StreamBigIntDeserializer() {} - - inline const void - deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { - auto flat = std::dynamic_pointer_cast>(vec); - flat->set(index, e.get_int64()); - } -}; - -struct StreamHugeIntDeserializer : public StreamJSONDeserializer { - public: - StreamHugeIntDeserializer() {} - - inline const void - deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { - auto flat = std::dynamic_pointer_cast>(vec); - flat->set(index, static_cast(e.get_int64())); - } -}; - -struct StreamStringDeserializer : public StreamJSONDeserializer { - public: - StreamStringDeserializer() {} - - inline const void - deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { - auto flat = - std::dynamic_pointer_cast>(vec); - std::string_view s = e.get_string(); - flat->set(index, facebook::velox::StringView(s.data(), s.size())); - } -}; - -struct StreamTimestampDeserializer : public StreamJSONDeserializer { - public: - StreamTimestampDeserializer() {} - - inline const void - deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { - auto flat = - std::dynamic_pointer_cast>(vec); - std::string_view s = e.get_string(); - const auto timestamp = - util::fromTimestampString( - s.data(), s.size(), util::TimestampParseMode::kLegacyCast) - .thenOrThrow(folly::identity, [&](const Status& status) { - VELOX_FAIL("error while parse timestamp: {}", status.message()); - }); - flat->set(index, timestamp); - } -}; - -struct StreamRowDeserializer : public StreamJSONDeserializer { - public: - StreamRowDeserializer( - const std::vector& fieldNames, - const std::vector& fieldTypes, - const std::vector>& deserializers) - : fieldNames_(fieldNames), - fieldTypes_(fieldTypes), - deserializers_(deserializers) {} - - inline const void - deserialize(JSONValue& e, const size_t index, VectorPtr& vec) override { - RowVectorPtr rowVector = std::dynamic_pointer_cast(vec); - std::vector& rowFields = rowVector->children(); - JSONRow row = e.get_object(); - for (size_t i = 0; i < fieldNames_.size(); ++i) { - JSONValue v; - auto err = row[fieldNames_[i]].get(v); - if (err != simdjson::error_code::SUCCESS || v.is_null()) { - rowFields[i]->setNull(index, true); - } else { - deserializers_[i]->deserialize(v, index, rowFields[i]); - } - } - } - - private: - std::vector fieldNames_; - std::vector fieldTypes_; - std::vector> deserializers_; -}; - -/// Class for kafka record deserialization of json format, using the streaming -/// interface of simdjson -class KafkaStreamJSONRecordDeserializer : public KafkaRecordDeserializer { - public: - KafkaStreamJSONRecordDeserializer( - const RowTypePtr& outputType, - memory::MemoryPool* memoryPool) - : KafkaRecordDeserializer(outputType, memoryPool), - deserializer_(std::dynamic_pointer_cast( - StreamJSONDeserializer::create(outputType))), - parser_(std::make_shared()) {} - - const void deserialize( - const std::string& message, - const size_t index, - VectorPtr& vec) override; - - private: - std::shared_ptr deserializer_; - std::shared_ptr parser_; -}; - -} // namespace facebook::velox::connector::kafka diff --git a/velox/connectors/kafka/tests/CMakeLists.txt b/velox/connectors/kafka/tests/CMakeLists.txt deleted file mode 100644 index 6a21de039f9e..000000000000 --- a/velox/connectors/kafka/tests/CMakeLists.txt +++ /dev/null @@ -1,27 +0,0 @@ -# Copyright (c) Facebook, Inc. and its affiliates. -# -# Licensed 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. -add_executable(velox_kafka_connector_test KafkaConnectorTest.cpp) - -add_test(velox_kafka_connector_test velox_kafka_connector_test) - -target_link_libraries( - velox_kafka_connector_test - velox_kafka_connector - velox_kafka_connector_format - velox_vector_test_lib - velox_exec_test_lib - velox_exec - velox_common_base - GTest::gtest - GTest::gtest_main) diff --git a/velox/connectors/kafka/tests/KafkaConnectorTest.cpp b/velox/connectors/kafka/tests/KafkaConnectorTest.cpp deleted file mode 100644 index 7bb01897d631..000000000000 --- a/velox/connectors/kafka/tests/KafkaConnectorTest.cpp +++ /dev/null @@ -1,163 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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 "velox/connectors/Connector.h" -#include "velox/connectors/kafka/KafkaDataSource.h" -#include "velox/connectors/kafka/format/StreamJSONRecordDeserializer.h" -#include "velox/connectors/kafka/tests/KafkaConnectorTestBase.h" -#include "velox/vector/ComplexVector.h" -#include "velox/type/Timestamp.h" -#include "velox/type/StringView.h" -#include -#include - -namespace facebook::velox::connector::kafka::test { - -class KafkaConnectorTest : public KafkaConnectorTestBase {}; - -TEST_F(KafkaConnectorTest, testConfig) { - std::shared_ptr kafkaConnector = getConnector(kKafkaConnectorId); - ASSERT_TRUE(kafkaConnector != nullptr); - const std::shared_ptr& connectorConfig = kafkaConnector->connectorConfig(); - const std::shared_ptr kafkaConfig = std::make_shared(connectorConfig); - ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kBootstrapServers)); - ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kTopic)); - ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kClientId)); - ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kGroupId)); - ASSERT_TRUE(kafkaConfig->exists(connector::kafka::ConnectionConfig::kFormat)); -} - -TEST_F(KafkaConnectorTest, testKafkaConsumer) { - const std::unique_ptr dataSource = createDataSource(); - KafkaDataSource * kafkaDataSource = reinterpret_cast(dataSource.get()); - ASSERT_TRUE(kafkaDataSource != nullptr); - const auto & kafkaConsumer = kafkaDataSource->getConsumer(); - ASSERT_TRUE(kafkaConsumer != nullptr); - std::vector topics = kafkaConsumer->getSubscribedTopics(); - ASSERT_TRUE(topics.size() > 0); - ASSERT_TRUE(topics[0] == kafkaTopic); -} - -TEST_F(KafkaConnectorTest, testAssignPartitions) { - const std::unique_ptr dataSource = createDataSource(); - KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); - ASSERT_TRUE(kafkaDataSource != nullptr); - kafkaDataSource->addSplit(createKafkaSplit()); - const auto & kafkaConsumer = kafkaDataSource->getConsumer(); - ASSERT_TRUE(kafkaConsumer != nullptr); - const cppkafka::TopicPartitionList tps = kafkaConsumer->getAssignedTopicPartitions(); - ASSERT_TRUE(tps.size() > 0); - const cppkafka::TopicPartition tp = tps[0]; - ASSERT_TRUE(tp.get_topic() == kafkaTopic); - ASSERT_TRUE(tp.get_partition() == 0); - ASSERT_TRUE(tp.get_offset() > 0); -} - -TEST_F(KafkaConnectorTest, testConsumeMessages) { - const std::unique_ptr dataSource = createDataSource(); - KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); - ASSERT_TRUE(kafkaDataSource != nullptr); - kafkaDataSource->addSplit(createKafkaSplit()); - std::string testMsg = "This is a test message!"; - sendMessageToKafka(testMsg); - const auto & kafkaConsumer = kafkaDataSource->getConsumer(); - ASSERT_TRUE(kafkaConsumer != nullptr); - std::vector msgs; - size_t msgBytes = 0; - kafkaConsumer->consumeBatch(msgs, msgBytes); - ASSERT_TRUE(msgs.size() == 1); - ASSERT_TRUE(msgs[0] == testMsg); - ASSERT_TRUE(msgBytes == testMsg.size()); -} - -TEST_F(KafkaConnectorTest, testCreateDeserializer) { - const std::unique_ptr dataSource = createDataSource(); - KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); - ASSERT_TRUE(kafkaDataSource != nullptr); - const auto & deserializer = kafkaDataSource->getDeserializer(); - const std::shared_ptr jsonDeserializer = - std::dynamic_pointer_cast(deserializer); - ASSERT_TRUE(jsonDeserializer != nullptr); -} - -TEST_F(KafkaConnectorTest, testDeserializeMessages) { - const std::unique_ptr dataSource = createDataSource(); - KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); - ASSERT_TRUE(kafkaDataSource != nullptr); - const auto & deserializer = kafkaDataSource->getDeserializer(); - std::string msg = "{\"event_type\":1, \"bid\": {\"auction\":1, \"bidder\":222, \"price\":1113, \"channel\":\"OTS\", \"url\":\"http://testkafka/a/b/c\", \"dateTime\":\"2025-06-18 11:22:33\", \"extra\":\"xxxx\"}}"; - VectorPtr vec1 = RowVector::createEmpty(outputType, memoryPool.get()); - vec1->resize(1); - deserializer->deserialize(msg, 0, vec1); - std::shared_ptr rowVector1 = std::dynamic_pointer_cast(vec1); - ASSERT_TRUE(rowVector1 != nullptr); - ASSERT_TRUE(rowVector1->size() == 1); - ASSERT_TRUE(rowVector1->children().size() == 2); - std::shared_ptr> flat = std::dynamic_pointer_cast>(rowVector1->childAt(0)); - ASSERT_TRUE(flat != nullptr); - ASSERT_TRUE(flat->size() == 1); - ASSERT_TRUE(flat->valueAt(0) == 1); - std::shared_ptr subRow = std::dynamic_pointer_cast(rowVector1->childAt(1)); - ASSERT_TRUE(subRow != nullptr); - ASSERT_TRUE(subRow->size() == 1); - ASSERT_TRUE(subRow->children().size() == 7); - std::shared_ptr> f1 = std::dynamic_pointer_cast>(subRow->childAt(0)); - std::shared_ptr> f2 = - std::dynamic_pointer_cast>(subRow->childAt(3)); - std::shared_ptr> f3 = - std::dynamic_pointer_cast>(subRow->childAt(5)); - ASSERT_TRUE(f1 != nullptr && f2 != nullptr && f3 != nullptr); - ASSERT_TRUE(f1->valueAt(0) == 1); - ASSERT_TRUE(f2->valueAt(0).str() == "OTS"); - ASSERT_TRUE(f3->valueAt(0).toMillis() == 1750245753000); -} - -TEST_F(KafkaConnectorTest, testKafkaSourceNext) { - const std::unique_ptr dataSource = createDataSource(); - KafkaDataSource* kafkaDataSource = reinterpret_cast(dataSource.get()); - ASSERT_TRUE(kafkaDataSource != nullptr); - kafkaDataSource->addSplit(createKafkaSplit()); - std::string msg1 = "{\"event_type\":1, \"bid\": {\"auction\":1, \"bidder\":222, \"price\":1113, \"channel\":\"OTS\", \"url\":\"http://testkafka/a/b/c\", \"dateTime\":\"2025-06-18 11:22:33\", \"extra\":\"xxxx\"}}"; - std::string msg2 = "{\"event_type\":2, \"bid\": {\"auction\":2, \"bidder\":223, \"price\":1114, \"channel\":\"OTS-1\", \"url\":\"http://testkafka/a/b/c\", \"dateTime\":\"2025-06-18 11:22:33\", \"extra\":\"xxxx\"}}"; - std::string msg3 = "{\"event_type\":3, \"bid\": {\"auction\":3, \"bidder\":225, \"price\":1115, \"channel\":\"OTS-2\", \"url\":\"http://testkafka/a/b/c\", \"dateTime\":\"2025-06-18 11:22:33\", \"extra\":\"xxxx\"}}"; - sendMessageToKafka(msg1); - sendMessageToKafka(msg2); - sendMessageToKafka(msg3); - auto future = facebook::velox::ContinueFuture{folly::Unit{}}; - std::optional res = kafkaDataSource->next(0, future); - ASSERT_TRUE(res.value() != nullptr); - RowVectorPtr rowVector = res.value(); - ASSERT_TRUE(rowVector->size() == 3); - std::shared_ptr> flat = std::dynamic_pointer_cast>(rowVector->childAt(0)); - std::shared_ptr subRow = std::dynamic_pointer_cast(rowVector->childAt(1)); - ASSERT_TRUE(flat != nullptr && flat->size() == 3); - ASSERT_TRUE(subRow != nullptr && subRow->size() == 3); - std::shared_ptr> subF1 = std::dynamic_pointer_cast>(subRow->childAt(0)); - ASSERT_TRUE(subF1 != nullptr && subF1->size() == 3); - for (size_t i = 0; i < 3; ++i) { - ASSERT_TRUE(flat->valueAt(i) == i+1); - ASSERT_TRUE(subF1->valueAt(i) == i+1); - } -} - -} // namespace facebook::velox::connector::kafka::test - -int main(int argc, char* argv[]) { - testing::InitGoogleTest(&argc, argv); - folly::Init init(&argc, &argv, false); - gflags::ParseCommandLineFlags(&argc, &argv, true); // Parse gflags - return RUN_ALL_TESTS(); -} diff --git a/velox/connectors/kafka/tests/KafkaConnectorTestBase.h b/velox/connectors/kafka/tests/KafkaConnectorTestBase.h deleted file mode 100644 index 3cd66a0946db..000000000000 --- a/velox/connectors/kafka/tests/KafkaConnectorTestBase.h +++ /dev/null @@ -1,184 +0,0 @@ -/* - * Copyright (c) Facebook, Inc. and its affiliates. - * - * Licensed 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. - */ - -#pragma once - -#include "velox/connectors/kafka/KafkaConnector.h" -#include "velox/connectors/kafka/KafkaConnectorSplit.h" -#include "velox/connectors/kafka/KafkaTableHandle.h" -#include "velox/connectors/kafka/KafkaConfig.h" -#include "velox/type/Type.h" -#include "velox/type/Filter.h" -#include "velox/exec/tests/utils/OperatorTestBase.h" -#include -#include - -namespace facebook::velox::connector::kafka::test { - -class KafkaConnectorTestBase : public exec::test::OperatorTestBase { - public: - std::string kafkaInstance = "localhost:9092"; - std::string kafkaTopic = "test_kafka"; - const std::string kafkaConsumeGroupId = "test_kafka_group_id"; - const std::string kafkaDataFormat = "json"; - const std::string kafkaClientId = "test_kafka_client"; - const std::string kafkaAutoOffsetReset = "earliest"; - const std::string kafkaStartupMode = "latest-offsets"; - const std::string kKafkaConnectorId = "test-kafka"; - const std::shared_ptr outputType = createOutputType(); - const std::shared_ptr memoryPool = memory::memoryManager()->addLeafPool(); - - void SetUp() override { - OperatorTestBase::SetUp(); - init(); - connector::registerConnectorFactory( - std::make_shared()); - std::unordered_map configMap; - configMap[connector::kafka::ConnectionConfig::kBootstrapServers] = kafkaInstance; - configMap[connector::kafka::ConnectionConfig::kGroupId] = kafkaConsumeGroupId; - configMap[connector::kafka::ConnectionConfig::kFormat] = kafkaDataFormat; - configMap[connector::kafka::ConnectionConfig::kClientId] = kafkaClientId; - configMap[connector::kafka::ConnectionConfig::kTopic] = kafkaTopic; - configMap[connector::kafka::ConnectionConfig::kAutoResetOffset] = kafkaAutoOffsetReset; - configMap[connector::kafka::ConnectionConfig::kStartupMode] = kafkaStartupMode; - std::shared_ptr config = - std::make_shared(std::move(configMap)); - auto kafkaConnector = - connector::getConnectorFactory( - connector::kafka::KafkaConnectorFactory::kKafkaConnectorName) - ->newConnector(kKafkaConnectorId, config); - connector::registerConnector(kafkaConnector); - } - - void TearDown() override { - connector::unregisterConnector(kKafkaConnectorId); - connector::unregisterConnectorFactory( - connector::kafka::KafkaConnectorFactory::kKafkaConnectorName); - OperatorTestBase::TearDown(); - } - - void init() { - char* currentPath; - if ((currentPath = getcwd(NULL, 0)) == NULL) { - VELOX_FAIL("Failed to get curent path"); - } - std::string kafkaConfName = "kafka.conf"; - char kafkaConfPath[strlen(currentPath) + kafkaConfName.size() + 1]; - sprintf(kafkaConfPath, "%s%s%s", currentPath, "/", kafkaConfName.data()); - std::ifstream kafkaConfFile(kafkaConfPath); - if (!kafkaConfFile.is_open()) { - VELOX_FAIL("Failed to open kafka config file: {}", std::string(kafkaConfPath, strlen(kafkaConfPath))); - } - std::string kKafkaTestInstance = "kafka.test.instance"; - std::string kKafkaTestTopic = "kafka.test.topic"; - std::string confLine; - while(getline(kafkaConfFile, confLine)) { - if (confLine.empty()) { - continue; - } - if (confLine.find(kKafkaTestInstance) != std::string::npos) { - kafkaInstance = confLine.substr(kKafkaTestInstance.size() + 1); - } else if (confLine.find(kKafkaTestTopic) != std::string::npos) { - kafkaTopic = confLine.substr(kKafkaTestTopic.size() + 1); - } - } - kafkaConfFile.close(); - } - - /// Row>> - static const std::shared_ptr createOutputType() { - std::vector bidRowFieldNames = {"auction", "bidder", "price", "channel", "url", "dateTime", "extra"}; - std::vector bidRowFieldTypes = { - std::make_shared(), - std::make_shared(), - std::make_shared(), - std::make_shared(), - std::make_shared(), - std::make_shared(), - std::make_shared() - }; - std::shared_ptr bidRowType = std::make_shared(std::move(bidRowFieldNames), std::move(bidRowFieldTypes)); - std::vector outputFieldNames = {"event_type", "bid"}; - std::vector outputFieldTypes = { - std::make_shared(), - bidRowType - }; - return std::make_shared(std::move(outputFieldNames), std::move(outputFieldTypes)); - } - - const std::shared_ptr createKafkaSplit() { - std::unordered_map>> topicPartitions; - std::vector> partitionOffsets { std::pair(0, 0) }; - topicPartitions[kafkaTopic] = partitionOffsets; - return std::make_shared( - kKafkaConnectorId, - kafkaInstance, - kafkaConsumeGroupId, - kafkaDataFormat, - false, - "earliest", - topicPartitions - ); - } - - const std::shared_ptr createKafkaTableHandle() { - return std::make_shared( - kKafkaConnectorId, - kafkaTopic, - outputType); - } - - const std::shared_ptr createQueryCtx() { - const auto kafkaConnector = getConnector(kKafkaConnectorId); - const auto connectorConfig = kafkaConnector->connectorConfig(); - std::shared_ptr connectorQueryCtx = - std::make_shared( - memoryPool.get(), - nullptr, - connectorConfig.get(), - nullptr, - common::PrefixSortConfig(), - nullptr, - nullptr, - "query.Kafka", - "task.Kafka", - "planNodeId.Kafka", - 0, - ""); - return connectorQueryCtx; - } - - const std::unique_ptr createDataSource() { - std::shared_ptr kafkaConnector = getConnector(kKafkaConnectorId); - const std::shared_ptr kafkaTableHandle = createKafkaTableHandle(); - std::unordered_map> columnHandles; - return kafkaConnector->createDataSource(outputType, kafkaTableHandle, columnHandles, createQueryCtx().get()); - } - - const void sendMessageToKafka(const std::string & message) { - cppkafka::Configuration config = {{"metadata.broker.list", kafkaInstance}}; - cppkafka::Producer producer(config); - cppkafka::MessageBuilder builder(kafkaTopic); - builder.partition(0); - builder.payload(message); - producer.produce(builder); - producer.flush(); - sleep(1); - } -}; - -} // namespace facebook::velox::connector::kafka::test diff --git a/velox/connectors/kafka/tests/kafka.conf b/velox/connectors/kafka/tests/kafka.conf deleted file mode 100644 index 748f310e103d..000000000000 --- a/velox/connectors/kafka/tests/kafka.conf +++ /dev/null @@ -1,2 +0,0 @@ -kafka.test.instance=localhost:9092 -kafka.test.topic=test_kafka diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 9f68b314d751..0b690f2274fa 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -13,14 +13,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "velox/experimental/stateful/WindowAggregator.h" -#include "velox/experimental/stateful/TimerHeapInternalTimer.h" #include "velox/type/Type.h" #include "velox/vector/ComplexVector.h" +#include "velox/experimental/stateful/WindowAggregator.h" +#include "velox/experimental/stateful/TimerHeapInternalTimer.h" #include "velox/experimental/stateful/window/TimeWindowUtil.h" - #include -#include #include namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/WindowAggregator.h b/velox/experimental/stateful/WindowAggregator.h index b06e65c46384..f2bcd50f3984 100644 --- a/velox/experimental/stateful/WindowAggregator.h +++ b/velox/experimental/stateful/WindowAggregator.h @@ -15,12 +15,8 @@ */ #pragma once -#include -#include -#include -#include -#include -#include +#include "velox/exec/Driver.h" +#include "velox/exec/Operator.h" #include "velox/experimental/stateful/InternalTimerService.h" #include "velox/experimental/stateful/KeySelector.h" #include "velox/experimental/stateful/StatefulOperator.h" @@ -28,6 +24,7 @@ #include "velox/experimental/stateful/Triggerable.h" #include "velox/experimental/stateful/window/SliceAssigner.h" #include "velox/experimental/stateful/window/WindowBuffer.h" +#include namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/window/MergingWindowSet.h b/velox/experimental/stateful/window/MergingWindowSet.h index ab727993bcad..3326ea9561d8 100644 --- a/velox/experimental/stateful/window/MergingWindowSet.h +++ b/velox/experimental/stateful/window/MergingWindowSet.h @@ -19,6 +19,7 @@ #include "velox/experimental/stateful/window/WindowProcessFunction.h" #include "velox/experimental/stateful/window/Window.h" #include +#include #include namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/window/WindowBuffer.h b/velox/experimental/stateful/window/WindowBuffer.h index a0ca2ea1f93c..20bfd7d5725e 100644 --- a/velox/experimental/stateful/window/WindowBuffer.h +++ b/velox/experimental/stateful/window/WindowBuffer.h @@ -18,6 +18,7 @@ #include "velox/experimental/stateful/window/WindowKey.h" #include "velox/vector/ComplexVector.h" #include +#include namespace facebook::velox::stateful { @@ -52,7 +53,6 @@ class RecordsWindowBuffer : public WindowBuffer { std::unordered_map> buffer_; // This is used to return empty map when no window is fired. std::unordered_map> empty_; - long minSliceEnd_ = LONG_MAX; int shiftTimeZone_ = 0; // TODO: support time zone shift }; From 8ec3318dcff396a5d67119cfc8b4c8eaeda7d50c Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Wed, 15 Oct 2025 11:57:23 +0000 Subject: [PATCH 05/15] remove useless changes --- CMakeLists.txt | 7 ------- velox/connectors/CMakeLists.txt | 1 - velox/connectors/fuzzer/DiscardDataSink.cpp | 1 - 3 files changed, 9 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index cbf6282ad595..45fd5bb56a1e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -128,7 +128,6 @@ set(VELOX_GFLAGS_TYPE option(VELOX_ENABLE_EXEC "Build exec." ON) option(VELOX_ENABLE_AGGREGATES "Build aggregates." ON) option(VELOX_ENABLE_HIVE_CONNECTOR "Build Hive connector." ON) -option(VELOX_ENABLE_KAFKA_CONNECTOR "Build Kafka connector." ON) option(VELOX_ENABLE_TPCH_CONNECTOR "Build TPC-H connector." ON) option(VELOX_ENABLE_PRESTO_FUNCTIONS "Build Presto SQL functions." ON) option(VELOX_ENABLE_SPARK_FUNCTIONS "Build Spark SQL functions." ON) @@ -176,7 +175,6 @@ if(${VELOX_BUILD_MINIMAL} OR ${VELOX_BUILD_MINIMAL_WITH_DWIO}) set(VELOX_ENABLE_EXEC OFF) set(VELOX_ENABLE_AGGREGATES OFF) set(VELOX_ENABLE_HIVE_CONNECTOR OFF) - set(VELOX_ENABLE_KAFKA_CONNECTOR ON) set(VELOX_ENABLE_TPCH_CONNECTOR OFF) set(VELOX_ENABLE_SPARK_FUNCTIONS OFF) set(VELOX_ENABLE_EXAMPLES OFF) @@ -190,11 +188,6 @@ if(${VELOX_ENABLE_BENCHMARKS}) set(VELOX_ENABLE_BENCHMARKS_BASIC ON) endif() -if(${VELOX_ENABLE_KAFKA_CONNECTOR}) - velox_set_source(CppKafka) - velox_resolve_dependency(CppKafka) -endif() - if(VELOX_ENABLE_BENCHMARKS_BASIC) set(VELOX_BUILD_TEST_UTILS ON) endif() diff --git a/velox/connectors/CMakeLists.txt b/velox/connectors/CMakeLists.txt index 032097691161..3c2c758cc41a 100644 --- a/velox/connectors/CMakeLists.txt +++ b/velox/connectors/CMakeLists.txt @@ -20,7 +20,6 @@ add_subdirectory(nexmark) add_subdirectory(print) add_subdirectory(from_elements) add_subdirectory(utils) -add_subdirectory(kafka) if(${VELOX_ENABLE_HIVE_CONNECTOR}) add_subdirectory(hive) diff --git a/velox/connectors/fuzzer/DiscardDataSink.cpp b/velox/connectors/fuzzer/DiscardDataSink.cpp index 40ab880f9ca7..7a11d060b646 100644 --- a/velox/connectors/fuzzer/DiscardDataSink.cpp +++ b/velox/connectors/fuzzer/DiscardDataSink.cpp @@ -32,7 +32,6 @@ void DiscardDataSink::appendData(RowVectorPtr input) { if (lastTime == 0) { lastTime = getCurrentTime(); } - LOG(INFO) << "input:" << input->toString(0); auto preNum = rowNums; rowNums += input->size(); if (rowNums / 100000 != preNum / 100000) { From 8afaa933e630680d068a6ece5f8abe58d8073093 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Thu, 16 Oct 2025 07:44:25 +0000 Subject: [PATCH 06/15] remove useless changes --- velox/experimental/stateful/InternalTimerService.h | 2 -- velox/experimental/stateful/WindowAggregator.cpp | 2 -- velox/experimental/stateful/WindowAggregator.h | 4 ++-- velox/experimental/stateful/window/SliceAssigner.cpp | 1 - 4 files changed, 2 insertions(+), 7 deletions(-) diff --git a/velox/experimental/stateful/InternalTimerService.h b/velox/experimental/stateful/InternalTimerService.h index 019735f57985..2f8ab477eada 100644 --- a/velox/experimental/stateful/InternalTimerService.h +++ b/velox/experimental/stateful/InternalTimerService.h @@ -20,9 +20,7 @@ #include #include #include -#include #include -#include #include namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 0b690f2274fa..9e5abd4036b6 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -74,7 +74,6 @@ void WindowAggregator::getOutput() { for (const auto& [sliceEnd, data] : sliceEndToData) { auto windowData = data; if (!isEventTime_) { - std::cout << "register process time:" << sliceEnd << " key:" << key << std::endl; windowTimerService_->registerProcessingTimeTimer(key, sliceEnd, sliceEnd); } @@ -238,7 +237,6 @@ void WindowAggregator::onEventTime(std::shared_ptr> timer) { - LOG(INFO) << "window agg on processing Time:" << lastTriggeredProcessingTime_ << " timer->timestamp():" << timer->timestamp(); if (timer->timestamp() >= lastTriggeredProcessingTime_) { lastTriggeredProcessingTime_ = timer->timestamp(); auto windowKeyToData = windowBuffer_->advanceProgress(timer->timestamp()); diff --git a/velox/experimental/stateful/WindowAggregator.h b/velox/experimental/stateful/WindowAggregator.h index f2bcd50f3984..ffb76dcf122c 100644 --- a/velox/experimental/stateful/WindowAggregator.h +++ b/velox/experimental/stateful/WindowAggregator.h @@ -71,7 +71,7 @@ class WindowAggregator : public StatefulOperator, public Triggerable - void clearWindow(K k, long timerTimestamp, long windowEnd); + void clearWindow(K key, long timerTimestamp, long windowEnd); std::unique_ptr localAggerator_; std::unique_ptr keySelector_; @@ -80,7 +80,7 @@ class WindowAggregator : public StatefulOperator, public Triggerable SliceAssigner::assignSliceEnd(const RowVectorPtr& input) { if (rowtimeIndex_ < 0) { // TODO: using Processing Time Service - // TODO: using TimeWindow.getWindowStartWithOffset to get window start, start + size as end long timestamp_ms = TimeWindowUtil::getCurrentProcessingTime(); if (windowType_ == 1) { // tumble window // TODO:: support get utcTimestamp by timezone. From 1a3c29c9a1addbcd83ba663b5220978d44717b59 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Tue, 16 Dec 2025 09:27:29 +0000 Subject: [PATCH 07/15] fix expr --- .../codegen/utils/resources/package.json | 35 ++++++++++++++++ .../stateful/GroupWindowAggregator.cpp | 34 +++++++-------- .../stateful/InternalPriorityQueue.h | 4 +- .../stateful/InternalTimerService.h | 26 ++++++------ velox/experimental/stateful/KeySelector.h | 2 +- .../stateful/LocalWindowAggregator.cpp | 4 +- .../stateful/ProcessingTimeService.h | 42 ++++++++++--------- velox/experimental/stateful/RuntimeContext.h | 6 +-- .../experimental/stateful/StatefulPlanner.cpp | 4 +- .../stateful/WindowAggregator.cpp | 35 +++++++--------- .../experimental/stateful/WindowAggregator.h | 6 +-- .../stateful/window/GroupWindowAssigner.cpp | 4 +- .../stateful/window/GroupWindowAssigner.h | 8 ++-- .../stateful/window/MergingWindowSet.cpp | 4 +- .../stateful/window/MergingWindowSet.h | 4 +- .../stateful/window/SliceAssigner.cpp | 27 ++++++------ .../stateful/window/SliceAssigner.h | 25 +++++------ .../stateful/window/TimeWindowUtil.cpp | 34 +++++++-------- .../stateful/window/TimeWindowUtil.h | 18 ++++---- velox/experimental/stateful/window/Window.h | 32 ++++++++++---- .../stateful/window/WindowBuffer.cpp | 4 +- .../stateful/window/WindowBuffer.h | 12 +++--- .../experimental/stateful/window/WindowKey.h | 8 ++-- .../window/WindowPartitionFunction.cpp | 12 +++--- .../stateful/window/WindowPartitionFunction.h | 26 ++++++------ .../stateful/window/WindowProcessFunction.cpp | 8 ++-- .../stateful/window/WindowProcessFunction.h | 24 +++++------ .../stateful/window/WindowTrigger.cpp | 8 ++-- .../stateful/window/WindowTrigger.h | 32 +++++++------- 29 files changed, 268 insertions(+), 220 deletions(-) create mode 100644 velox/experimental/codegen/utils/resources/package.json diff --git a/velox/experimental/codegen/utils/resources/package.json b/velox/experimental/codegen/utils/resources/package.json new file mode 100644 index 000000000000..7836fb807a15 --- /dev/null +++ b/velox/experimental/codegen/utils/resources/package.json @@ -0,0 +1,35 @@ + + {"optimizationLevel": "-O3", + "extraLinkOptions" : ["-Wl,-undefined,dynamic_lookup", "-Wl,-rpath", "." ], + "extraCompileOption": [ + "-g", + "-v", + "-std=c++17", + "-mavx2", + "-mfma", + "-mavx", + "-mf16c", + "-march=native", + "-DUSE_VELOX_COMMON_BASE", + "-mllvm", + "-inline-threshold=5000", + "-DBOOST_ALL_NO_LIB", + "-DBOOST_CONTEXT_DYN_LINK", + "-DBOOST_REGEX_DYN_LINK", + "-fPIC", + , + ], + "extraLinkOptions" : [], + "defaultLibraries": [ + { + "includePath" : ["/home/zouyunhe/velox4j/src/main/cpp"], + "additionalLinkerObject": [] + } + , { + "includePath" : ["/usr/local/include" ,"/usr/include/zlib.h" ,"/usr/local/include/folly" ,"/usr/include"] + }], + "compilerPath": "/usr/bin/c++", + "linker": "/usr/bin/ld", + "tempDirectory": "/tmp" + } + \ No newline at end of file diff --git a/velox/experimental/stateful/GroupWindowAggregator.cpp b/velox/experimental/stateful/GroupWindowAggregator.cpp index 62bc47615342..a862adf73566 100644 --- a/velox/experimental/stateful/GroupWindowAggregator.cpp +++ b/velox/experimental/stateful/GroupWindowAggregator.cpp @@ -24,7 +24,7 @@ GroupWindowAggregator::GroupWindowAggregator( std::vector> targets, std::unique_ptr keySelector, std::unique_ptr sliceAssigner, - long allowedLateness, + int64_t allowedLateness, bool produceUpdates, int rowtimeIndex, bool isEventTime, @@ -143,11 +143,11 @@ void GroupWindowAggregator::close() { } void GroupWindowAggregator::registerCleanupTimer(uint32_t key, TimeWindow window) { - long cleanupTime = + int64_t cleanupTime = TimeWindowUtil::toEpochMillsForTimer( TimeWindowUtil::cleanupTime(window.maxTimestamp(), allowedLateness_, isEventTime_), shiftTimeZone_); - if (cleanupTime == LONG_MAX) { + if (cleanupTime == INT64_MAX) { // don't set a GC timer for "end of time" return; } @@ -190,17 +190,17 @@ void GroupWindowAggregator::WindowTriggerContext::open() { } bool GroupWindowAggregator::WindowTriggerContext::onElement( - uint32_t key, RowVectorPtr row, long timestamp, TimeWindow window) { + uint32_t key, RowVectorPtr row, int64_t timestamp, TimeWindow window) { return trigger_->onElement(key, row, timestamp, window); } bool GroupWindowAggregator::WindowTriggerContext::onProcessingTime( - TimeWindow window, long time) { + TimeWindow window, int64_t time) { return trigger_->onProcessingTime(window, time); } bool GroupWindowAggregator::WindowTriggerContext::onEventTime( - TimeWindow window, long time) { + TimeWindow window, int64_t time) { return trigger_->onEventTime(window, time); } @@ -209,31 +209,31 @@ void GroupWindowAggregator::WindowTriggerContext::onMerge( trigger_->onMerge(key, window, nullptr); } -long GroupWindowAggregator::WindowTriggerContext::getCurrentProcessingTime() { +int64_t GroupWindowAggregator::WindowTriggerContext::getCurrentProcessingTime() { return internalTimerService_->currentProcessingTime(); } -long GroupWindowAggregator::WindowTriggerContext::getCurrentWatermark() { +int64_t GroupWindowAggregator::WindowTriggerContext::getCurrentWatermark() { return internalTimerService_->currentWatermark(); } void GroupWindowAggregator::WindowTriggerContext::registerProcessingTimeTimer( - uint32_t key, TimeWindow window, long time) { + uint32_t key, TimeWindow window, int64_t time) { internalTimerService_->registerProcessingTimeTimer(key, window, time); } void GroupWindowAggregator::WindowTriggerContext::registerEventTimeTimer( - uint32_t key, TimeWindow window, long time) { + uint32_t key, TimeWindow window, int64_t time) { internalTimerService_->registerEventTimeTimer(key, window, time); } void GroupWindowAggregator::WindowTriggerContext::deleteProcessingTimeTimer( - uint32_t key, TimeWindow window, long time) { + uint32_t key, TimeWindow window, int64_t time) { internalTimerService_->deleteProcessingTimeTimer(key, window, time); } void GroupWindowAggregator::WindowTriggerContext::deleteEventTimeTimer( - uint32_t key, TimeWindow window, long time) { + uint32_t key, TimeWindow window, int64_t time) { internalTimerService_->deleteEventTimeTimer(key, window, time); } @@ -265,7 +265,7 @@ WindowContext::WindowContext( std::shared_ptr stateHandler, int shiftTimeZone, bool isEventTime, - long allowedLateness) + int64_t allowedLateness) : windowAggregator_(windowAggregator), windowState_(std::move(windowState)), timerService_(std::move(timerService)), @@ -280,11 +280,11 @@ StatePtr WindowContext::getPartitionedState(StateDescriptor& stateDescriptor) { return stateHandler_->getGroupMapState(stateDescriptor); } -long WindowContext::currentProcessingTime() { +int64_t WindowContext::currentProcessingTime() { return timerService_->currentProcessingTime(); } -long WindowContext::currentWatermark() { +int64_t WindowContext::currentWatermark() { return timerService_->currentWatermark(); } @@ -316,11 +316,11 @@ void WindowContext::clearTrigger(TimeWindow window) { } void WindowContext::deleteCleanupTimer(TimeWindow window) { - long cleanupTime = + int64_t cleanupTime = TimeWindowUtil::toEpochMillsForTimer( TimeWindowUtil::cleanupTime(window.maxTimestamp(), allowedLateness_, isEventTime_), shiftTimeZone_); - if (cleanupTime == LONG_MAX) { + if (cleanupTime == INT64_MAX) { // no need to clean up because we didn't set one return; } diff --git a/velox/experimental/stateful/InternalPriorityQueue.h b/velox/experimental/stateful/InternalPriorityQueue.h index 41a21042999b..7c312b82b923 100644 --- a/velox/experimental/stateful/InternalPriorityQueue.h +++ b/velox/experimental/stateful/InternalPriorityQueue.h @@ -32,7 +32,7 @@ class InternalPriorityQueue { virtual void clear() = 0; - virtual bool empty() = 0; + virtual bool empty() const = 0; }; // This class is relevent to flink HeapPriorityQueue. @@ -75,7 +75,7 @@ class HeapPriorityQueue : public InternalPriorityQueue { size_ = 0; } - bool empty() override { + bool empty() const override { return size_ == 0; } diff --git a/velox/experimental/stateful/InternalTimerService.h b/velox/experimental/stateful/InternalTimerService.h index 2f8ab477eada..69b8ca0c4f5b 100644 --- a/velox/experimental/stateful/InternalTimerService.h +++ b/velox/experimental/stateful/InternalTimerService.h @@ -32,33 +32,33 @@ class InternalTimerService { InternalTimerService(Triggerable* triggerable) : triggerable_(triggerable), processingTimeService_(std::make_shared()) {} - void registerEventTimeTimer(K key, N ns, long time) { + void registerEventTimeTimer(K key, N ns, int64_t time) { eventTimeTimersQueue_.add(std::make_shared>(time, key, ns)); } - void deleteEventTimeTimer(K key, N ns, long time) { + void deleteEventTimeTimer(K key, N ns, int64_t time) { eventTimeTimersQueue_.remove(std::make_shared>(time, key, ns)); } - void registerProcessingTimeTimer(K key, N ns, long time) { + void registerProcessingTimeTimer(K key, N ns, int64_t time) { std::shared_ptr> oldHead = processingTimeTimersQueue_.peek(); processingTimeTimersQueue_.add(std::make_shared>(time, key, ns)); - long nextTriggerTime = oldHead != nullptr ? oldHead->timestamp() : std::numeric_limits::max() ; + int64_t nextTriggerTime = oldHead != nullptr ? oldHead->timestamp() : std::numeric_limits::max() ; if (time < nextTriggerTime) { if (nextTimer_.has_value()) { processingTimeService_->cancel(nextTimer_.value()); } - nextTimer_ = processingTimeService_->registerTimer(time, ProcessingTimerTask(time, [&](long processingTime) { + nextTimer_ = processingTimeService_->registerTimer(time, ProcessingTimerTask(time, [&](int64_t processingTime) { onProcessingTime(processingTime); })); } } - void deleteProcessingTimeTimer(K key, N ns, long time) { + void deleteProcessingTimeTimer(K key, N ns, int64_t time) { processingTimeTimersQueue_.remove(std::make_shared>(time, key, ns)); } - long currentWatermark() { + int64_t currentWatermark() { // TODO: Implement watermark logic if needed. if (eventTimeTimersQueue_.peek() != nullptr) { return eventTimeTimersQueue_.peek()->timestamp(); @@ -66,12 +66,12 @@ class InternalTimerService { return 0; // or some other default value } - long currentProcessingTime() { + int64_t currentProcessingTime() { // TODO: Implement processing time logic if needed. return 0; // or some other default value } - void advanceWatermark(long time) { + void advanceWatermark(int64_t time) { while (eventTimeTimersQueue_.peek() != nullptr && eventTimeTimersQueue_.peek()->timestamp() <= time) { auto timer = eventTimeTimersQueue_.poll(); @@ -86,7 +86,7 @@ class InternalTimerService { } private: - void onProcessingTime(long time) { + void onProcessingTime(int64_t time) { std::string taskName = ""; if (nextTimer_.has_value()) { taskName = nextTimer_.value(); @@ -106,11 +106,11 @@ class InternalTimerService { } if (!taskName.empty()) { - processingTimeService_->finish(taskName); + processingTimeService_->unregister(taskName); } if (timer != nullptr && !nextTimer_.has_value()) { - nextTimer_ = processingTimeService_->registerTimer(timer->timestamp(), ProcessingTimerTask(timer->timestamp(), [&](long processingTime) { + nextTimer_ = processingTimeService_->registerTimer(timer->timestamp(), ProcessingTimerTask(timer->timestamp(), [&](int64_t processingTime) { onProcessingTime(processingTime); })); } @@ -118,7 +118,7 @@ class InternalTimerService { Triggerable* triggerable_; std::optional nextTimer_; - std::shared_ptr processingTimeService_; + std::shared_ptr processingTimeService_; HeapPriorityQueue>> eventTimeTimersQueue_; HeapPriorityQueue>> processingTimeTimersQueue_; }; diff --git a/velox/experimental/stateful/KeySelector.h b/velox/experimental/stateful/KeySelector.h index a9328b3ee9ec..d64a6ea507b3 100644 --- a/velox/experimental/stateful/KeySelector.h +++ b/velox/experimental/stateful/KeySelector.h @@ -29,7 +29,7 @@ class KeySelector { KeySelector( std::unique_ptr partitionFunction, memory::MemoryPool* pool, - int numPartitions = INT_MAX); + int numPartitions = 1024); std::map partition(const RowVectorPtr& input); diff --git a/velox/experimental/stateful/LocalWindowAggregator.cpp b/velox/experimental/stateful/LocalWindowAggregator.cpp index 6c55a308fee7..eb7bb082ed66 100644 --- a/velox/experimental/stateful/LocalWindowAggregator.cpp +++ b/velox/experimental/stateful/LocalWindowAggregator.cpp @@ -23,7 +23,7 @@ LocalWindowAggregator::LocalWindowAggregator( std::vector> targets, std::unique_ptr keySelector, std::unique_ptr sliceAssigner, - const long windowInterval, + const int64_t windowInterval, const bool useDayLightSaving, RowTypePtr outputType) : StatefulOperator(std::move(op), std::move(targets)), @@ -59,7 +59,7 @@ void LocalWindowAggregator::getOutput() { input_.reset(); } -void LocalWindowAggregator::processWatermarkInternal(long timestamp) { +void LocalWindowAggregator::processWatermarkInternal(int64_t timestamp) { if (timestamp > currentWatermark_) { currentWatermark_ = timestamp; if (currentWatermark_ >= nextTriggerWatermark_) { diff --git a/velox/experimental/stateful/ProcessingTimeService.h b/velox/experimental/stateful/ProcessingTimeService.h index b08de25e44ee..88df1c0ca4bd 100644 --- a/velox/experimental/stateful/ProcessingTimeService.h +++ b/velox/experimental/stateful/ProcessingTimeService.h @@ -17,17 +17,21 @@ #include "velox/experimental/stateful/window/TimeWindowUtil.h" #include +#include +#include +#include #include #include +#include namespace facebook::velox::stateful { -using ProcessingTimeCallback = std::function; +using ProcessingTimeCallback = std::function; class ProcessingTimerTask { public: ProcessingTimerTask( - long time, + int64_t time, ProcessingTimeCallback callback) : time_(time), callback_(callback) {} @@ -35,53 +39,52 @@ class ProcessingTimerTask { callback_(time_); } private: - long time_; + int64_t time_; ProcessingTimeCallback callback_; }; -class ProcessingTimeSerivice { +class ProcessingTimeService { public: - long getCurrentProcessingTime() { + int64_t getCurrentProcessingTime() { return TimeWindowUtil::getCurrentProcessingTime(); } - virtual std::optional registerTimer(long timestamp, ProcessingTimerTask target) { - std::optional task; - return task; - } + virtual std::optional registerTimer(int64_t timestamp, ProcessingTimerTask target) = 0; virtual void cancel(const std::string& task) {} virtual void close() {} - void finish(const std::string& task) { + void unregister(const std::string& task) { auto it = std::find(registry.begin(), registry.end(), task); if (it != registry.end()) { registry.erase(it); } } - std::string generateTimerTaskName(long timestamp) { - return "proc_time_task_" + std::to_string(timestamp); + std::string generateTimerTaskName(int64_t timestamp) { + boost::uuids::random_generator generator; + boost::uuids::uuid uuid = generator(); + return "proc_time_task_" + std::to_string(timestamp) + "-" + to_string(uuid); } protected: - std::vector registry; + std::set registry; }; -class SystemProcessingTimeService : public ProcessingTimeSerivice { +class SystemProcessingTimeService : public ProcessingTimeService { public: - SystemProcessingTimeService() : ProcessingTimeSerivice() { + SystemProcessingTimeService() : ProcessingTimeService() { executor_ = std::make_shared(); executor_->start(); } - std::optional registerTimer(long timestamp, ProcessingTimerTask task) override { - long currentTimestamp = getCurrentProcessingTime(); - long delay = 0; + std::optional registerTimer(int64_t timestamp, ProcessingTimerTask task) override { + int64_t currentTimestamp = getCurrentProcessingTime(); + int64_t delay = 0; if (timestamp >= currentTimestamp) { delay = timestamp - currentTimestamp; } std::string taskName = generateTimerTaskName(timestamp); if (delay > 0) { executor_->addFunctionOnce(task, taskName, std::chrono::microseconds(delay * 1000)); - registry.emplace_back(taskName); + registry.emplace(taskName); } else { task(); } @@ -100,6 +103,7 @@ class SystemProcessingTimeService : public ProcessingTimeSerivice { if (executor_) { executor_->shutdown(); } + registry.clear(); } private: std::shared_ptr executor_; diff --git a/velox/experimental/stateful/RuntimeContext.h b/velox/experimental/stateful/RuntimeContext.h index af4c8471a954..0af7722d8156 100644 --- a/velox/experimental/stateful/RuntimeContext.h +++ b/velox/experimental/stateful/RuntimeContext.h @@ -32,17 +32,17 @@ class RuntimeContext { return keyedStateBackend_->getOrCreateMapState(stateDescriptor); } - std::shared_ptr> getListState( + std::shared_ptr> getListState( StateDescriptor& stateDescriptor) { return keyedStateBackend_->getOrCreateListState(stateDescriptor); } - std::shared_ptr> getValueState( + std::shared_ptr> getValueState( StateDescriptor& stateDescriptor) { return keyedStateBackend_->getOrCreateValueState(stateDescriptor); } - std::shared_ptr> createTimerService( + std::shared_ptr> createTimerService( Triggerable* triggerable) { return keyedStateBackend_->createTimerService(triggerable); } diff --git a/velox/experimental/stateful/StatefulPlanner.cpp b/velox/experimental/stateful/StatefulPlanner.cpp index da14a8f522a1..8dd2e9db5a65 100644 --- a/velox/experimental/stateful/StatefulPlanner.cpp +++ b/velox/experimental/stateful/StatefulPlanner.cpp @@ -179,7 +179,7 @@ StatefulOperatorPtr StatefulPlanner::nodeToStatefulOperator( windowAggNode->size(), windowAggNode->step(), windowAggNode->offset(), - windowAggNode->windowType(), + Window::getType(windowAggNode->windowType()), windowAggNode->rowtimeIndex()); return std::make_unique( windowAggNode->isEventTime() ? std::move(localAggregator) : nullptr, @@ -210,7 +210,7 @@ StatefulOperatorPtr StatefulPlanner::nodeToStatefulOperator( 0, 0, 0, - windowAggNode->windowType(), + Window::getType(windowAggNode->windowType()), windowAggNode->rowtimeIndex()); return std::make_unique( std::unique_ptr(dynamic_cast(op.release())), diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 9e5abd4036b6..5ca30ffa747f 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -24,8 +24,8 @@ namespace facebook::velox::stateful { WindowAggregator::WindowAggregator( - std::unique_ptr localAggerator, - std::unique_ptr globalAggerator, + std::unique_ptr localAggregator, + std::unique_ptr globalAggregator, std::vector> targets, std::unique_ptr keySelector, std::unique_ptr sliceAssigner, @@ -34,8 +34,8 @@ WindowAggregator::WindowAggregator( const bool isEventTime, const int windowStartIndex, const int windowEndIndex) - : StatefulOperator(std::move(globalAggerator), std::move(targets)), - localAggerator_(std::move(localAggerator)), + : StatefulOperator(std::move(globalAggregator), std::move(targets)), + localAggregator_(std::move(localAggregator)), keySelector_(std::move(keySelector)), sliceAssigner_(std::move(sliceAssigner)), windowInterval_(windowInterval), @@ -48,8 +48,8 @@ WindowAggregator::WindowAggregator( void WindowAggregator::initialize() { StatefulOperator::initialize(); - if (localAggerator_) { - localAggerator_->initialize(); + if (localAggregator_) { + localAggregator_->initialize(); } StateDescriptor stateDesc("window-aggs"); @@ -117,8 +117,8 @@ void WindowAggregator::processWatermarkInternal(long timestamp) { continue; } // TODO: agg should output no matter how many rows in datas. - localAggerator_->addInput(TimeWindowUtil::mergeVectors(datas, op()->pool())); - RowVectorPtr localAcc = localAggerator_->getOutput(); + localAggregator_->addInput(TimeWindowUtil::mergeVectors(datas, op()->pool())); + RowVectorPtr localAcc = localAggregator_->getOutput(); auto stateAcc = windowState_->value(windowKey.key(), windowKey.window()); std::list allDatas; if (!localAcc && !stateAcc) { @@ -149,7 +149,6 @@ void WindowAggregator::processWatermarkInternal(long timestamp) { RowVectorPtr addWindowTimestampToOutput( const RowVectorPtr& output, const std::string& fieldName, - const TypePtr& fieldType, const long fieldValue, const int fieldIndex) { auto createTimestampVector = [&]( @@ -178,7 +177,7 @@ RowVectorPtr addWindowTimestampToOutput( newOutputFieldNames.emplace_back(outputFieldNames[i]); newOutputFields.emplace_back(outputFields[i]); } - newOutputFieldTypes.emplace_back(fieldType); + newOutputFieldTypes.emplace_back(std::make_shared()); newOutputFieldNames.emplace_back(fieldName); newOutputFields.emplace_back(windowStartVec); for (int i = fieldIndex + 1; i < output->childrenSize() + 1; ++i) { @@ -204,12 +203,14 @@ void WindowAggregator::onTimer(std::shared_ptr void WindowAggregator::fireWindow(K key, long timerTimestamp, long windowEnd) { RowVectorPtr output = windowState_->value(key, windowEnd); - if (output) { + if (!output) { + LOG(INFO) << "No output found for key: " << key << ", window end: " << windowEnd; + return; + } else { if (windowStartIndex_ >= 0) { output = addWindowTimestampToOutput( output, "window_start", - std::make_shared(), windowEnd - windowInterval_, windowStartIndex_); } @@ -217,7 +218,6 @@ void WindowAggregator::fireWindow(K key, long timerTimestamp, long windowEnd) { output = addWindowTimestampToOutput( output, "window_end", - std::make_shared(), windowEnd, windowEndIndex_); } @@ -244,10 +244,7 @@ void WindowAggregator::onProcessingTime(std::shared_ptr allDatas; - for (const auto& data: datas) { - allDatas.push_back(data); - } + std::list allDatas(datas.begin(), datas.end()); auto stateAcc = windowState_->value(windowKey.key(), windowKey.window()); if (stateAcc) { allDatas.push_back(stateAcc); @@ -272,8 +269,8 @@ long WindowAggregator::sliceStateMergeTarget(long sliceToMerge) { void WindowAggregator::close() { processWatermarkInternal(INT_MAX); StatefulOperator::close(); - if (localAggerator_) { - localAggerator_->close(); + if (localAggregator_) { + localAggregator_->close(); } input_.reset(); windowBuffer_->clear(); diff --git a/velox/experimental/stateful/WindowAggregator.h b/velox/experimental/stateful/WindowAggregator.h index ffb76dcf122c..f31f9b44948e 100644 --- a/velox/experimental/stateful/WindowAggregator.h +++ b/velox/experimental/stateful/WindowAggregator.h @@ -33,8 +33,8 @@ namespace facebook::velox::stateful { class WindowAggregator : public StatefulOperator, public Triggerable { public: WindowAggregator( - std::unique_ptr localAggerator, - std::unique_ptr globalAggerator, + std::unique_ptr localAggregator, + std::unique_ptr globalAggregator, std::vector> targets, std::unique_ptr keySelector, std::unique_ptr sliceAssigner, @@ -73,7 +73,7 @@ class WindowAggregator : public StatefulOperator, public Triggerable void clearWindow(K key, long timerTimestamp, long windowEnd); - std::unique_ptr localAggerator_; + std::unique_ptr localAggregator_; std::unique_ptr keySelector_; std::unique_ptr sliceAssigner_; WindowBufferPtr windowBuffer_; diff --git a/velox/experimental/stateful/window/GroupWindowAssigner.cpp b/velox/experimental/stateful/window/GroupWindowAssigner.cpp index c5c5c40a177b..27aa04507aa5 100644 --- a/velox/experimental/stateful/window/GroupWindowAssigner.cpp +++ b/velox/experimental/stateful/window/GroupWindowAssigner.cpp @@ -18,12 +18,12 @@ namespace facebook::velox::stateful { SessionWindowAssigner::SessionWindowAssigner( - long gap, + int64_t gap, bool isEventTime) : gap_(gap), isEventTime_(isEventTime) { } -std::vector SessionWindowAssigner::assignWindows(RowVectorPtr element, long timestamp) { +std::vector SessionWindowAssigner::assignWindows(RowVectorPtr element, int64_t timestamp) { return {TimeWindow(timestamp, timestamp + gap_)}; } diff --git a/velox/experimental/stateful/window/GroupWindowAssigner.h b/velox/experimental/stateful/window/GroupWindowAssigner.h index fd7b573d4b56..4672c3470e23 100644 --- a/velox/experimental/stateful/window/GroupWindowAssigner.h +++ b/velox/experimental/stateful/window/GroupWindowAssigner.h @@ -27,7 +27,7 @@ namespace facebook::velox::stateful { template>> class GroupWindowAssigner { public: - virtual std::vector assignWindows(RowVectorPtr element, long timestamp) = 0; + virtual std::vector assignWindows(RowVectorPtr element, int64_t timestamp) = 0; virtual bool isEventTime() = 0; }; @@ -42,9 +42,9 @@ using GroupWindowAssignerPtr = std::shared_ptr>; class SessionWindowAssigner : public MergingWindowAssigner { public: - SessionWindowAssigner(long gap, bool isEventTime); + SessionWindowAssigner(int64_t gap, bool isEventTime); - std::vector assignWindows(RowVectorPtr element, long timestamp) override; + std::vector assignWindows(RowVectorPtr element, int64_t timestamp) override; void mergeWindows( TimeWindow newWindow, std::set& sortedWindows, MergeResultCollector& callback); @@ -57,7 +57,7 @@ class SessionWindowAssigner : public MergingWindowAssigner { TimeWindow mergeWindow( const TimeWindow& curWindow, const TimeWindow& other, std::set& mergedWindow); - long gap_; + int64_t gap_; bool isEventTime_; }; diff --git a/velox/experimental/stateful/window/MergingWindowSet.cpp b/velox/experimental/stateful/window/MergingWindowSet.cpp index 77215c9c5fec..69736405a678 100644 --- a/velox/experimental/stateful/window/MergingWindowSet.cpp +++ b/velox/experimental/stateful/window/MergingWindowSet.cpp @@ -123,7 +123,7 @@ void MergingWindowSet::close() { MergingFunction::MergingFunction( std::shared_ptr accMergingConsumer, std::shared_ptr> ctx, - long allowedLateness, + int64_t allowedLateness, bool isEventTime) : accMergingConsumer_(std::move(accMergingConsumer)), ctx_(ctx), @@ -137,7 +137,7 @@ void MergingFunction::merge( TimeWindow stateWindowResult, std::vector& stateWindowsToBeMerged) { - long mergeResultMaxTs = + int64_t mergeResultMaxTs = TimeWindowUtil::toEpochMillsForTimer(mergeResult.maxTimestamp(), ctx_->getShiftTimeZone()); VELOX_CHECK(!(isEventTime_ && mergeResultMaxTs + allowedLateness_ <= ctx_->currentWatermark()), diff --git a/velox/experimental/stateful/window/MergingWindowSet.h b/velox/experimental/stateful/window/MergingWindowSet.h index 3326ea9561d8..f34988ecc74b 100644 --- a/velox/experimental/stateful/window/MergingWindowSet.h +++ b/velox/experimental/stateful/window/MergingWindowSet.h @@ -63,7 +63,7 @@ class MergingFunction { MergingFunction( std::shared_ptr accMergingConsumer, std::shared_ptr> ctx, - long allowedLateness, + int64_t allowedLateness, bool isEventTime); void merge( @@ -75,7 +75,7 @@ class MergingFunction { private: std::shared_ptr accMergingConsumer_; std::shared_ptr> ctx_; - long allowedLateness_; + int64_t allowedLateness_; bool isEventTime_; }; diff --git a/velox/experimental/stateful/window/SliceAssigner.cpp b/velox/experimental/stateful/window/SliceAssigner.cpp index e604597d7028..26db67bfb70d 100644 --- a/velox/experimental/stateful/window/SliceAssigner.cpp +++ b/velox/experimental/stateful/window/SliceAssigner.cpp @@ -15,19 +15,16 @@ */ #include "velox/experimental/stateful/window/SliceAssigner.h" #include "velox/experimental/stateful/window/TimeWindowUtil.h" - -#include #include -#include namespace facebook::velox::stateful { SliceAssigner::SliceAssigner( std::unique_ptr keySelector, - long size, - long step, - long offset, - int windowType, + int64_t size, + int64_t step, + int64_t offset, + WindowType windowType, int rowtimeIndex) : keySelector_(std::move(keySelector)), size_(size), @@ -42,11 +39,11 @@ SliceAssigner::SliceAssigner( std::map SliceAssigner::assignSliceEnd(const RowVectorPtr& input) { if (rowtimeIndex_ < 0) { // TODO: using Processing Time Service - long timestamp_ms = TimeWindowUtil::getCurrentProcessingTime(); - if (windowType_ == 1) { // tumble window + int64_t timestamp_ms = TimeWindowUtil::getCurrentProcessingTime(); + if (windowType_ == WindowType::TUMBLE) { // tumble window // TODO:: support get utcTimestamp by timezone. - long utcTimestamp = TimeWindowUtil::toEpochMillsForTimer(timestamp_ms, 0); - long windowStart = stateful::TimeWindowUtil::getWindowStartWithOffset(utcTimestamp, offset_, size_); + int64_t utcTimestamp = TimeWindowUtil::toEpochMillsForTimer(timestamp_ms, 0); + int64_t windowStart = stateful::TimeWindowUtil::getWindowStartWithOffset(utcTimestamp, offset_, size_); return {{windowStart + size_, input}}; } else { return {{timestamp_ms, input}}; @@ -55,18 +52,18 @@ std::map SliceAssigner::assignSliceEnd(const RowVectorPt return keySelector_->partition(input); } -long SliceAssigner::getLastWindowEnd(long sliceEnd) { - if (windowType_ == 0) { // Hopping window +int64_t SliceAssigner::getLastWindowEnd(int64_t sliceEnd) { + if (windowType_ == WindowType::HOP) { // Hopping window return sliceEnd - sliceSize_ + size_; } return sliceEnd; } -long SliceAssigner::getWindowStart(long windowEnd) { +int64_t SliceAssigner::getWindowStart(int64_t windowEnd) { return windowEnd - size_; } -long SliceAssigner::getSliceEndInterval() { +int64_t SliceAssigner::getSliceEndInterval() { return sliceSize_; } diff --git a/velox/experimental/stateful/window/SliceAssigner.h b/velox/experimental/stateful/window/SliceAssigner.h index 5d2ac294c869..13023503a7f9 100644 --- a/velox/experimental/stateful/window/SliceAssigner.h +++ b/velox/experimental/stateful/window/SliceAssigner.h @@ -16,6 +16,7 @@ #pragma once #include "velox/experimental/stateful/KeySelector.h" +#include "velox/experimental/stateful/window/Window.h" namespace facebook::velox::stateful { @@ -24,30 +25,30 @@ class SliceAssigner { public: SliceAssigner( std::unique_ptr keySelector, - long size, - long step, - long offset, - int windowType, + int64_t size, + int64_t step, + int64_t offset, + WindowType windowType, int rowtimeIndex); std::map assignSliceEnd(const RowVectorPtr& input); - long getLastWindowEnd(long sliceEnd); + int64_t getLastWindowEnd(int64_t sliceEnd); - long getWindowStart(long windowEnd); + int64_t getWindowStart(int64_t windowEnd); // Iterable expiredSlices(long windowEnd); - long getSliceEndInterval(); + int64_t getSliceEndInterval(); private: const std::unique_ptr keySelector_; - const long size_; - const long step_; - const long offset_; - const int windowType_; // 0: hopping window, 1: slide window - long sliceSize_; + const int64_t size_; + const int64_t step_; + const int64_t offset_; + const WindowType windowType_; + int64_t sliceSize_; int rowtimeIndex_; }; diff --git a/velox/experimental/stateful/window/TimeWindowUtil.cpp b/velox/experimental/stateful/window/TimeWindowUtil.cpp index d2c9c376228c..342c5bf58ca8 100644 --- a/velox/experimental/stateful/window/TimeWindowUtil.cpp +++ b/velox/experimental/stateful/window/TimeWindowUtil.cpp @@ -20,25 +20,25 @@ namespace facebook::velox::stateful { // static -long TimeWindowUtil::getNextTriggerWatermark( - long currentWatermark, - long interval, +int64_t TimeWindowUtil::getNextTriggerWatermark( + int64_t currentWatermark, + int64_t interval, int shiftTimezone, bool useDayLightSaving) { - if (currentWatermark == LONG_MAX) { + if (currentWatermark == INT64_MAX) { return currentWatermark; } - long triggerWatermark; + int64_t triggerWatermark; // consider the DST timezone if (useDayLightSaving) { // TODO: support time zone - //long utcWindowStart = + //int64_t utcWindowStart = // getWindowStartWithOffset( // toUtcTimestampMills(currentWatermark, shiftTimezone), 0L, interval); //triggerWatermark = toEpochMillsForTimer(utcWindowStart + interval - 1, shiftTimezone); } else { - long start = getWindowStartWithOffset(currentWatermark, 0L, interval); + int64_t start = getWindowStartWithOffset(currentWatermark, 0L, interval); triggerWatermark = start + interval - 1; } @@ -50,8 +50,8 @@ long TimeWindowUtil::getNextTriggerWatermark( } // static -long TimeWindowUtil::getWindowStartWithOffset(long timestamp, long offset, long windowSize) { - long remainder = (timestamp - offset) % windowSize; +int64_t TimeWindowUtil::getWindowStartWithOffset(int64_t timestamp, int64_t offset, int64_t windowSize) { + int64_t remainder = (timestamp - offset) % windowSize; // handle both positive and negative cases if (remainder < 0) { return timestamp - (remainder + windowSize); @@ -60,34 +60,34 @@ long TimeWindowUtil::getWindowStartWithOffset(long timestamp, long offset, long } } -long TimeWindowUtil::getCurrentProcessingTime() { +int64_t TimeWindowUtil::getCurrentProcessingTime() { auto now = std::chrono::system_clock::now(); return std::chrono::duration_cast(now.time_since_epoch()).count(); } // static bool TimeWindowUtil::isWindowFired( - long windowEnd, long currentProgress, int shiftTimeZone) { - if (windowEnd == LONG_MAX) { + int64_t windowEnd, int64_t currentProgress, int shiftTimeZone) { + if (windowEnd == INT64_MAX) { return false; } // TODO: support time zone - long windowTriggerTime = toEpochMillsForTimer(windowEnd - 1, shiftTimeZone); + int64_t windowTriggerTime = toEpochMillsForTimer(windowEnd - 1, shiftTimeZone); return currentProgress >= windowTriggerTime; } // static -long TimeWindowUtil::cleanupTime(long maxTimestamp, long allowedLateness, bool isEventTime) { +int64_t TimeWindowUtil::cleanupTime(int64_t maxTimestamp, int64_t allowedLateness, bool isEventTime) { if (isEventTime) { - long cleanupTime = std::max(0L, maxTimestamp + allowedLateness); - return cleanupTime >= maxTimestamp ? cleanupTime : LONG_MAX; + int64_t cleanupTime = std::max(0L, maxTimestamp + allowedLateness); + return cleanupTime >= maxTimestamp ? cleanupTime : INT64_MAX; } else { return std::max(0L, maxTimestamp); } } // static -long TimeWindowUtil::toEpochMillsForTimer(long timestamp, int shiftTimeZone) { +int64_t TimeWindowUtil::toEpochMillsForTimer(int64_t timestamp, int shiftTimeZone) { // TODO: support time zone return timestamp; } diff --git a/velox/experimental/stateful/window/TimeWindowUtil.h b/velox/experimental/stateful/window/TimeWindowUtil.h index 3cb1fabb3c88..2e5cf8ccd435 100644 --- a/velox/experimental/stateful/window/TimeWindowUtil.h +++ b/velox/experimental/stateful/window/TimeWindowUtil.h @@ -17,34 +17,32 @@ #include "velox/vector/ComplexVector.h" #include -#include -#include namespace facebook::velox::stateful { // This class is relevent to flink TimeWindowUitl. class TimeWindowUtil { public: - static long getNextTriggerWatermark( - long currentWatermark, - long interval, + static int64_t getNextTriggerWatermark( + int64_t currentWatermark, + int64_t interval, int shiftTimezone, bool useDayLightSaving); - static long getWindowStartWithOffset(long timestamp, long offset, long windowSize); + static int64_t getWindowStartWithOffset(int64_t timestamp, int64_t offset, int64_t windowSize); static bool isWindowFired( - long windowEnd, long currentProgress, int shiftTimeZone); + int64_t windowEnd, int64_t currentProgress, int shiftTimeZone); static RowVectorPtr mergeVectors( const std::list& vectors, memory::MemoryPool* pool); - static long toEpochMillsForTimer(long timestamp, int shiftTimeZone); + static int64_t toEpochMillsForTimer(int64_t timestamp, int shiftTimeZone); - static long cleanupTime(long maxTimestamp, long allowedLateness_, bool isEventTime); + static int64_t cleanupTime(int64_t maxTimestamp, int64_t allowedLateness_, bool isEventTime); - static long getCurrentProcessingTime(); + static int64_t getCurrentProcessingTime(); }; diff --git a/velox/experimental/stateful/window/Window.h b/velox/experimental/stateful/window/Window.h index 1ca7d8027e9b..58a7b605aefe 100644 --- a/velox/experimental/stateful/window/Window.h +++ b/velox/experimental/stateful/window/Window.h @@ -15,19 +15,35 @@ */ #pragma once +#include #include #include namespace facebook::velox::stateful { +enum class WindowType : int { + HOP = 0, + TUMBLE, + SESSION, + CUMULATIVE +}; + // This class is relevent to flink WindowBuffer. class Window { public: - virtual long maxTimestamp() = 0; + virtual int64_t maxTimestamp() = 0; virtual bool operator<(const Window& other) const = 0; virtual std::string toString() const = 0; + + static WindowType getType(const int32_t t) { + if (t >= 0 && t <= 3) { + return static_cast(t); + } else { + VELOX_FAIL("Window type value {} is illegal, it is not between 0 and 3", t); + } + } }; class TimeWindow : public Window { @@ -35,18 +51,18 @@ class TimeWindow : public Window { TimeWindow() : start_(-1), end_(-1) {} - TimeWindow(long start, long end) + TimeWindow(int64_t start, int64_t end) : start_(start), end_(end) {} - long maxTimestamp() override { + int64_t maxTimestamp() override { return end_ - 1; } - long start() const { + int64_t start() const { return start_; } - long end() const { + int64_t end() const { return end_; } @@ -79,8 +95,8 @@ class TimeWindow : public Window { } private: - long start_; - long end_; + int64_t start_; + int64_t end_; }; } // namespace facebook::velox::stateful @@ -90,7 +106,7 @@ template<> struct hash { size_t operator()(const facebook::velox::stateful::TimeWindow& w) const { // TODO: verify it. - return hash()(w.start()) ^ (hash()(w.end()) << 1); + return hash()(w.start()) ^ (hash()(w.end()) << 1); } }; } \ No newline at end of file diff --git a/velox/experimental/stateful/window/WindowBuffer.cpp b/velox/experimental/stateful/window/WindowBuffer.cpp index 7713d368a50c..50bbcead02a5 100644 --- a/velox/experimental/stateful/window/WindowBuffer.cpp +++ b/velox/experimental/stateful/window/WindowBuffer.cpp @@ -20,7 +20,7 @@ namespace facebook::velox::stateful { -void RecordsWindowBuffer::addElement(uint32_t key, long sliceEnd, RowVectorPtr& element) { +void RecordsWindowBuffer::addElement(uint32_t key, int64_t sliceEnd, RowVectorPtr& element) { minSliceEnd_ = std::min(sliceEnd, minSliceEnd_); WindowKey windowKey(key, sliceEnd); auto it = buffer_.find(windowKey); @@ -35,7 +35,7 @@ void RecordsWindowBuffer::addElement(uint32_t key, long sliceEnd, RowVectorPtr& } } -std::unordered_map>& RecordsWindowBuffer::advanceProgress(long progress) { +std::unordered_map>& RecordsWindowBuffer::advanceProgress(int64_t progress) { if (TimeWindowUtil::isWindowFired(minSliceEnd_, progress, shiftTimeZone_)) { // there should be some window to be fired, flush buffer to state first return buffer_; diff --git a/velox/experimental/stateful/window/WindowBuffer.h b/velox/experimental/stateful/window/WindowBuffer.h index 20bfd7d5725e..b4a90836a664 100644 --- a/velox/experimental/stateful/window/WindowBuffer.h +++ b/velox/experimental/stateful/window/WindowBuffer.h @@ -27,9 +27,9 @@ class WindowBuffer { public: // TODO: we use hash key of RowVector as key, but flink use RowVector as key. // This is not equal to flink, should check it. - virtual void addElement(uint32_t key, long window, RowVectorPtr& element) = 0; + virtual void addElement(uint32_t key, int64_t window, RowVectorPtr& element) = 0; - virtual std::unordered_map>& advanceProgress(long progress) = 0; + virtual std::unordered_map>& advanceProgress(int64_t progress) = 0; virtual void clear() = 0; }; @@ -39,13 +39,13 @@ using WindowBufferPtr = std::shared_ptr; // This class is relevent to flink RecordsWindowBuffer. class RecordsWindowBuffer : public WindowBuffer { public: - void addElement(uint32_t key, long sliceEnd, RowVectorPtr& element) override; + void addElement(uint32_t key, int64_t sliceEnd, RowVectorPtr& element) override; - std::unordered_map>& advanceProgress(long progress) override; + std::unordered_map>& advanceProgress(int64_t progress) override; void clear() override { buffer_.clear(); - minSliceEnd_ = LONG_MAX; + minSliceEnd_ = INT64_MAX; } private: @@ -53,7 +53,7 @@ class RecordsWindowBuffer : public WindowBuffer { std::unordered_map> buffer_; // This is used to return empty map when no window is fired. std::unordered_map> empty_; - long minSliceEnd_ = LONG_MAX; + int64_t minSliceEnd_ = INT64_MAX; int shiftTimeZone_ = 0; // TODO: support time zone shift }; diff --git a/velox/experimental/stateful/window/WindowKey.h b/velox/experimental/stateful/window/WindowKey.h index 8245c17abee4..b6f22cef1655 100644 --- a/velox/experimental/stateful/window/WindowKey.h +++ b/velox/experimental/stateful/window/WindowKey.h @@ -24,10 +24,10 @@ namespace facebook::velox::stateful { // This class is relevent to flink WindowBuffer. class WindowKey { public: - WindowKey(uint32_t key, long window) + WindowKey(uint32_t key, int64_t window) : key_(key), window_(window) {} - long window() const { + int64_t window() const { return window_; } @@ -41,7 +41,7 @@ class WindowKey { private: uint32_t key_; - long window_; + int64_t window_; }; } // namespace facebook::velox::stateful @@ -51,7 +51,7 @@ template<> struct hash { size_t operator()(const facebook::velox::stateful::WindowKey& key) const { // TODO: RowVector should have a hash function. - return std::hash()(key.window()) ^ std::hash()(key.key()); + return std::hash()(key.window()) ^ std::hash()(key.key()); } }; } diff --git a/velox/experimental/stateful/window/WindowPartitionFunction.cpp b/velox/experimental/stateful/window/WindowPartitionFunction.cpp index 1a74bce58d07..a0baceb799de 100644 --- a/velox/experimental/stateful/window/WindowPartitionFunction.cpp +++ b/velox/experimental/stateful/window/WindowPartitionFunction.cpp @@ -24,9 +24,9 @@ namespace facebook::velox::stateful { WindowPartitionFunction::WindowPartitionFunction( const RowTypePtr& inputType, const column_index_t rowtimeIndex, - long size, - long step, - long offset, + int64_t size, + int64_t step, + int64_t offset, int windowType) : inputType_(std::move(inputType)), rowtimeIndex_(rowtimeIndex), @@ -57,9 +57,9 @@ std::optional WindowPartitionFunction::partition( for (auto i = 0; i < size; ++i) { auto child = input.childAt(rowtimeIndex_); auto ts = child->as>()->valueAt(i); - long timestamp = ts.getSeconds() * 1'000 + ts.getNanos() / 1'000'000; + int64_t timestamp = ts.getSeconds() * 1'000 + ts.getNanos() / 1'000'000; if (windowType_ == 0) { // Hopping window - long start = TimeWindowUtil::getWindowStartWithOffset(timestamp, offset_, sliceSize_); + int64_t start = TimeWindowUtil::getWindowStartWithOffset(timestamp, offset_, sliceSize_); partitions[i] = start + sliceSize_; } else if (windowType_ == 1) { // Windowed Slice Assigner partitions[i] = timestamp; @@ -71,7 +71,7 @@ std::optional WindowPartitionFunction::partition( return std::nullopt; } -long getTimestamp( +int64_t getTimestamp( const RowVectorPtr& input, RowTypePtr inputType, const column_index_t rowtimeIndex ) { diff --git a/velox/experimental/stateful/window/WindowPartitionFunction.h b/velox/experimental/stateful/window/WindowPartitionFunction.h index e85de1e408bd..0feb56747a1d 100644 --- a/velox/experimental/stateful/window/WindowPartitionFunction.h +++ b/velox/experimental/stateful/window/WindowPartitionFunction.h @@ -27,9 +27,9 @@ class WindowPartitionFunction : public core::PartitionFunction { WindowPartitionFunction( const RowTypePtr& inputType, const column_index_t rowtimeIndex, - long size, - long step, - long offset, + int64_t size, + int64_t step, + int64_t offset, int windowType); std::optional partition( @@ -39,11 +39,11 @@ class WindowPartitionFunction : public core::PartitionFunction { private: RowTypePtr inputType_; column_index_t rowtimeIndex_; - long size_; - long step_; - long offset_; + int64_t size_; + int64_t step_; + int64_t offset_; int windowType_; - long sliceSize_; + int64_t sliceSize_; }; class StreamWindowPartitionFunctionSpec : public core::PartitionFunctionSpec { @@ -51,9 +51,9 @@ class StreamWindowPartitionFunctionSpec : public core::PartitionFunctionSpec { StreamWindowPartitionFunctionSpec( const RowTypePtr& inputType, column_index_t rowtimeIndex, - long size, - long step, - long offset, + int64_t size, + int64_t step, + int64_t offset, int windowType) : inputType_(std::move(inputType)), rowtimeIndex_(rowtimeIndex), @@ -77,9 +77,9 @@ class StreamWindowPartitionFunctionSpec : public core::PartitionFunctionSpec { private: RowTypePtr inputType_; column_index_t rowtimeIndex_; - long size_; - long step_; - long offset_; + int64_t size_; + int64_t step_; + int64_t offset_; int windowType_; }; diff --git a/velox/experimental/stateful/window/WindowProcessFunction.cpp b/velox/experimental/stateful/window/WindowProcessFunction.cpp index 8034978477ad..210a692add00 100644 --- a/velox/experimental/stateful/window/WindowProcessFunction.cpp +++ b/velox/experimental/stateful/window/WindowProcessFunction.cpp @@ -21,7 +21,7 @@ namespace facebook::velox::stateful { MergingWindowProcessFunction::MergingWindowProcessFunction( std::shared_ptr windowAssigner, exec::Operator* windowAggregator, - long allowedLateness) + int64_t allowedLateness) : WindowProcessFunction(windowAssigner, windowAggregator, allowedLateness) { } @@ -43,7 +43,7 @@ void MergingWindowProcessFunction::open(std::shared_ptr MergingWindowProcessFunction::assignStateNamespace( - uint32_t key, RowVectorPtr inputRow, long timestamp) { + uint32_t key, RowVectorPtr inputRow, int64_t timestamp) { std::vector elementWindows = windowAssigner_->assignWindows(inputRow, timestamp); mergingWindows_->initializeCache(key); reuseActualWindows_.clear(); @@ -68,11 +68,11 @@ std::vector MergingWindowProcessFunction::assignStateNamespace( } std::vector MergingWindowProcessFunction::assignActualWindows( - RowVectorPtr inputRow, long timestamp) { + RowVectorPtr inputRow, int64_t timestamp) { return reuseActualWindows_; } -void MergingWindowProcessFunction::cleanWindowIfNeeded(TimeWindow window, long currentTime) { +void MergingWindowProcessFunction::cleanWindowIfNeeded(TimeWindow window, int64_t currentTime) { if (isCleanupTime(window, currentTime)) { ctx_->clearTrigger(window); TimeWindow stateWindow = mergingWindows_->getStateWindow(ctx_->currentKey(), window); diff --git a/velox/experimental/stateful/window/WindowProcessFunction.h b/velox/experimental/stateful/window/WindowProcessFunction.h index f1ac8f7bb585..abe455851076 100644 --- a/velox/experimental/stateful/window/WindowProcessFunction.h +++ b/velox/experimental/stateful/window/WindowProcessFunction.h @@ -36,9 +36,9 @@ class FunctionContext { virtual uint32_t currentKey() = 0; - virtual long currentProcessingTime() = 0; + virtual int64_t currentProcessingTime() = 0; - virtual long currentWatermark() = 0; + virtual int64_t currentWatermark() = 0; virtual int getShiftTimeZone() = 0; @@ -64,7 +64,7 @@ class WindowProcessFunction { WindowProcessFunction( GroupWindowAssignerPtr windowAssigner, exec::Operator* windowAggregator, - long allowedLateness) + int64_t allowedLateness) : windowAssigner_(windowAssigner), windowAggregator_(dynamic_cast(windowAggregator)), allowedLateness_(allowedLateness) {} @@ -73,14 +73,14 @@ class WindowProcessFunction { ctx_ = std::move(ctx); } - virtual std::vector assignStateNamespace(uint32_t key, RowVectorPtr inputRow, long timestamp) = 0; + virtual std::vector assignStateNamespace(uint32_t key, RowVectorPtr inputRow, int64_t timestamp) = 0; - virtual std::vector assignActualWindows(RowVectorPtr inputRow, long timestamp) = 0; + virtual std::vector assignActualWindows(RowVectorPtr inputRow, int64_t timestamp) = 0; // TODO: implement it when necessary. virtual void prepareAggregateAccumulatorForEmit(uint32_t key, W window) = 0; - virtual void cleanWindowIfNeeded(W window, long currentTime) = 0; + virtual void cleanWindowIfNeeded(W window, int64_t currentTime) = 0; // TODO: implement it when necessary. virtual void close() {} @@ -97,7 +97,7 @@ class WindowProcessFunction { <= ctx_->currentWatermark())); } - bool isCleanupTime(W window, long time) { + bool isCleanupTime(W window, int64_t time) { return time == TimeWindowUtil::toEpochMillsForTimer( TimeWindowUtil::cleanupTime( window.maxTimestamp(), allowedLateness_, windowAssigner_->isEventTime()), @@ -107,7 +107,7 @@ class WindowProcessFunction { GroupWindowAssignerPtr windowAssigner_; // TODO: windowAggregator may need state GroupWindowAggsHandler* windowAggregator_; - long allowedLateness_; + int64_t allowedLateness_; std::shared_ptr> ctx_; }; @@ -121,17 +121,17 @@ class MergingWindowProcessFunction : public WindowProcessFunction { MergingWindowProcessFunction( std::shared_ptr windowAssigner, exec::Operator* windowAggregator, - long allowedLateness); + int64_t allowedLateness); void open(std::shared_ptr> ctx) override; std::vector - assignStateNamespace(uint32_t key, RowVectorPtr inputRow, long timestamp) override; + assignStateNamespace(uint32_t key, RowVectorPtr inputRow, int64_t timestamp) override; std::vector - assignActualWindows(RowVectorPtr inputRow, long timestamp) override; + assignActualWindows(RowVectorPtr inputRow, int64_t timestamp) override; - void cleanWindowIfNeeded(TimeWindow window, long currentTime) override; + void cleanWindowIfNeeded(TimeWindow window, int64_t currentTime) override; void prepareAggregateAccumulatorForEmit(uint32_t key, TimeWindow window) override; diff --git a/velox/experimental/stateful/window/WindowTrigger.cpp b/velox/experimental/stateful/window/WindowTrigger.cpp index 738b3c162d79..537c865e817c 100644 --- a/velox/experimental/stateful/window/WindowTrigger.cpp +++ b/velox/experimental/stateful/window/WindowTrigger.cpp @@ -18,7 +18,7 @@ namespace facebook::velox::stateful { -long WindowTrigger::triggerTime(TimeWindow window) { +int64_t WindowTrigger::triggerTime(TimeWindow window) { return TimeWindowUtil::toEpochMillsForTimer(window.maxTimestamp(), ctx_->getShiftTimeZone()); } @@ -26,7 +26,7 @@ void AfterEndOfWindow::open(std::shared_ptr ctx) { ctx_ = ctx; } -bool AfterEndOfWindow::onElement(uint32_t key, RowVectorPtr element, long timestamp, TimeWindow window) { +bool AfterEndOfWindow::onElement(uint32_t key, RowVectorPtr element, int64_t timestamp, TimeWindow window) { if (triggerTime(window) <= ctx_->getCurrentWatermark()) { // if the watermark is already past the window fire immediately return true; @@ -36,11 +36,11 @@ bool AfterEndOfWindow::onElement(uint32_t key, RowVectorPtr element, long timest } } -bool AfterEndOfWindow::onProcessingTime(TimeWindow window, long time) { +bool AfterEndOfWindow::onProcessingTime(TimeWindow window, int64_t time) { return false; } -bool AfterEndOfWindow::onEventTime(TimeWindow window, long time) { +bool AfterEndOfWindow::onEventTime(TimeWindow window, int64_t time) { return time == triggerTime(window); } diff --git a/velox/experimental/stateful/window/WindowTrigger.h b/velox/experimental/stateful/window/WindowTrigger.h index 8cfb7e2c3631..acaabe9422b8 100644 --- a/velox/experimental/stateful/window/WindowTrigger.h +++ b/velox/experimental/stateful/window/WindowTrigger.h @@ -29,11 +29,11 @@ class WindowTrigger { public: virtual void open(std::shared_ptr ctx) = 0; - virtual bool onElement(uint32_t key, RowVectorPtr element, long timestamp, TimeWindow window) = 0; + virtual bool onElement(uint32_t key, RowVectorPtr element, int64_t timestamp, TimeWindow window) = 0; - virtual bool onProcessingTime(TimeWindow window, long time) = 0; + virtual bool onProcessingTime(TimeWindow window, int64_t time) = 0; - virtual bool onEventTime(TimeWindow window, long time) = 0; + virtual bool onEventTime(TimeWindow window, int64_t time) = 0; virtual bool canMerge() { return false; @@ -45,7 +45,7 @@ class WindowTrigger { virtual void clear(uint32_t key, TimeWindow window) = 0; - virtual long triggerTime(TimeWindow window); + virtual int64_t triggerTime(TimeWindow window); protected: std::shared_ptr ctx_; @@ -55,11 +55,11 @@ class AfterEndOfWindow : public WindowTrigger { public: void open(std::shared_ptr ctx); - bool onElement(uint32_t key, RowVectorPtr element, long timestamp, TimeWindow window) override; + bool onElement(uint32_t key, RowVectorPtr element, int64_t timestamp, TimeWindow window) override; - bool onProcessingTime(TimeWindow window, long time) override; + bool onProcessingTime(TimeWindow window, int64_t time) override; - bool onEventTime(TimeWindow window, long time) override ; + bool onEventTime(TimeWindow window, int64_t time) override ; void clear(uint32_t key, TimeWindow window) override; @@ -72,28 +72,28 @@ class TriggerContext : public std::enable_shared_from_this{ public: virtual void open() = 0; - virtual bool onElement(uint32_t key, RowVectorPtr row, long timestamp, TimeWindow window) = 0; + virtual bool onElement(uint32_t key, RowVectorPtr row, int64_t timestamp, TimeWindow window) = 0; - virtual bool onProcessingTime(TimeWindow window, long time) = 0; + virtual bool onProcessingTime(TimeWindow window, int64_t time) = 0; - virtual bool onEventTime(TimeWindow window, long time) = 0; + virtual bool onEventTime(TimeWindow window, int64_t time) = 0; virtual void onMerge(uint32_t key, TimeWindow window) = 0; - virtual long getCurrentProcessingTime() = 0; + virtual int64_t getCurrentProcessingTime() = 0; - virtual long getCurrentWatermark() = 0; + virtual int64_t getCurrentWatermark() = 0; // TODO: support it // MetricGroup getMetricGroup()ï¼› - virtual void registerProcessingTimeTimer(uint32_t key, TimeWindow window, long time) = 0; + virtual void registerProcessingTimeTimer(uint32_t key, TimeWindow window, int64_t time) = 0; - virtual void registerEventTimeTimer(uint32_t key, TimeWindow window, long time) = 0; + virtual void registerEventTimeTimer(uint32_t key, TimeWindow window, int64_t time) = 0; - virtual void deleteProcessingTimeTimer(uint32_t key, TimeWindow window, long time) = 0; + virtual void deleteProcessingTimeTimer(uint32_t key, TimeWindow window, int64_t time) = 0; - virtual void deleteEventTimeTimer(uint32_t key, TimeWindow window, long time) = 0; + virtual void deleteEventTimeTimer(uint32_t key, TimeWindow window, int64_t time) = 0; virtual int getShiftTimeZone() = 0; From 665ac096ff49637ad8c00d1f9eb6353eed3b92b4 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Tue, 16 Dec 2025 09:29:22 +0000 Subject: [PATCH 08/15] remove useless changes --- .../codegen/utils/resources/package.json | 35 ------------------- 1 file changed, 35 deletions(-) delete mode 100644 velox/experimental/codegen/utils/resources/package.json diff --git a/velox/experimental/codegen/utils/resources/package.json b/velox/experimental/codegen/utils/resources/package.json deleted file mode 100644 index 7836fb807a15..000000000000 --- a/velox/experimental/codegen/utils/resources/package.json +++ /dev/null @@ -1,35 +0,0 @@ - - {"optimizationLevel": "-O3", - "extraLinkOptions" : ["-Wl,-undefined,dynamic_lookup", "-Wl,-rpath", "." ], - "extraCompileOption": [ - "-g", - "-v", - "-std=c++17", - "-mavx2", - "-mfma", - "-mavx", - "-mf16c", - "-march=native", - "-DUSE_VELOX_COMMON_BASE", - "-mllvm", - "-inline-threshold=5000", - "-DBOOST_ALL_NO_LIB", - "-DBOOST_CONTEXT_DYN_LINK", - "-DBOOST_REGEX_DYN_LINK", - "-fPIC", - , - ], - "extraLinkOptions" : [], - "defaultLibraries": [ - { - "includePath" : ["/home/zouyunhe/velox4j/src/main/cpp"], - "additionalLinkerObject": [] - } - , { - "includePath" : ["/usr/local/include" ,"/usr/include/zlib.h" ,"/usr/local/include/folly" ,"/usr/include"] - }], - "compilerPath": "/usr/bin/c++", - "linker": "/usr/bin/ld", - "tempDirectory": "/tmp" - } - \ No newline at end of file From 873a0eb63d8ba61966259b6a51ad4f5e154b1f9e Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Tue, 16 Dec 2025 09:36:07 +0000 Subject: [PATCH 09/15] fix 111 --- velox/experimental/stateful/WindowAggregator.cpp | 8 ++++---- velox/experimental/stateful/WindowAggregator.h | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 5ca30ffa747f..e7796ed9feab 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -13,13 +13,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +#include +#include #include "velox/type/Type.h" #include "velox/vector/ComplexVector.h" #include "velox/experimental/stateful/WindowAggregator.h" #include "velox/experimental/stateful/TimerHeapInternalTimer.h" #include "velox/experimental/stateful/window/TimeWindowUtil.h" -#include -#include namespace facebook::velox::stateful { @@ -149,7 +149,7 @@ void WindowAggregator::processWatermarkInternal(long timestamp) { RowVectorPtr addWindowTimestampToOutput( const RowVectorPtr& output, const std::string& fieldName, - const long fieldValue, + const int64_t fieldValue, const int fieldIndex) { auto createTimestampVector = [&]( const Timestamp& val, @@ -267,7 +267,7 @@ long WindowAggregator::sliceStateMergeTarget(long sliceToMerge) { } void WindowAggregator::close() { - processWatermarkInternal(INT_MAX); + processWatermarkInternal(std::numeric_limits::max()); StatefulOperator::close(); if (localAggregator_) { localAggregator_->close(); diff --git a/velox/experimental/stateful/WindowAggregator.h b/velox/experimental/stateful/WindowAggregator.h index f31f9b44948e..cf9945b3cfa6 100644 --- a/velox/experimental/stateful/WindowAggregator.h +++ b/velox/experimental/stateful/WindowAggregator.h @@ -15,6 +15,7 @@ */ #pragma once +#include #include "velox/exec/Driver.h" #include "velox/exec/Operator.h" #include "velox/experimental/stateful/InternalTimerService.h" @@ -24,7 +25,6 @@ #include "velox/experimental/stateful/Triggerable.h" #include "velox/experimental/stateful/window/SliceAssigner.h" #include "velox/experimental/stateful/window/WindowBuffer.h" -#include namespace facebook::velox::stateful { From e216597b629056388b883030d9de23a152fbe7f3 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Fri, 19 Dec 2025 03:47:04 +0000 Subject: [PATCH 10/15] fix compile err --- velox/experimental/stateful/window/Window.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/velox/experimental/stateful/window/Window.h b/velox/experimental/stateful/window/Window.h index 58a7b605aefe..3672a81940d8 100644 --- a/velox/experimental/stateful/window/Window.h +++ b/velox/experimental/stateful/window/Window.h @@ -15,7 +15,7 @@ */ #pragma once -#include +#include "velox/common/base/Exceptions.h" #include #include From a1123b6afc2fd231bf0e7b601b29334bb4d05007 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Tue, 23 Dec 2025 03:23:20 +0000 Subject: [PATCH 11/15] fix proctime window trigger mutilple times for one key --- .../stateful/GroupWindowAggregator.cpp | 4 +- .../stateful/InternalPriorityQueue.h | 67 +++++++++++++++++-- .../stateful/InternalTimerService.h | 19 +++--- velox/experimental/stateful/KeySelector.cpp | 8 +-- velox/experimental/stateful/KeySelector.h | 4 +- .../stateful/LocalWindowAggregator.cpp | 5 +- .../stateful/StreamKeyedOperator.cpp | 2 - .../stateful/TimerHeapInternalTimer.h | 25 +++++-- .../stateful/WatermarkAssigner.cpp | 2 - .../stateful/WindowAggregator.cpp | 7 +- .../stateful/window/SliceAssigner.cpp | 2 +- .../stateful/window/SliceAssigner.h | 2 +- .../window/WindowPartitionFunction.cpp | 5 +- 13 files changed, 111 insertions(+), 41 deletions(-) diff --git a/velox/experimental/stateful/GroupWindowAggregator.cpp b/velox/experimental/stateful/GroupWindowAggregator.cpp index a862adf73566..42ff24bb85f4 100644 --- a/velox/experimental/stateful/GroupWindowAggregator.cpp +++ b/velox/experimental/stateful/GroupWindowAggregator.cpp @@ -82,12 +82,12 @@ void GroupWindowAggregator::getOutput() { } // 1. Partition input by key - std::map keyToData = keySelector_->partition(input_); + std::map keyToData = keySelector_->partition(input_); for (const auto& [key, keyedData] : keyToData) { // 2. Set the current key in the context windowContext_->setCurrentKey(key); // 3. Partition the keyed data by rowtime or processing time - std::map timestampToData = sliceAssigner_->assignSliceEnd(keyedData); + std::map timestampToData = sliceAssigner_->assignSliceEnd(keyedData); for (const auto& [timestamp, data] : timestampToData) { // 4. Assign data to window std::vector windows = diff --git a/velox/experimental/stateful/InternalPriorityQueue.h b/velox/experimental/stateful/InternalPriorityQueue.h index 7c312b82b923..58bbbe6ac48d 100644 --- a/velox/experimental/stateful/InternalPriorityQueue.h +++ b/velox/experimental/stateful/InternalPriorityQueue.h @@ -24,7 +24,7 @@ namespace facebook::velox::stateful { template class InternalPriorityQueue { public: - virtual void add(T toAdd) = 0; + virtual bool add(T toAdd) = 0; virtual T poll() = 0; @@ -33,9 +33,11 @@ class InternalPriorityQueue { virtual void clear() = 0; virtual bool empty() const = 0; + + virtual void remove(T toRemove) = 0; }; -// This class is relevent to flink HeapPriorityQueue. +// This class is relevant to flink HeapPriorityQueue. // TODO: need to make it equal to flink template class HeapPriorityQueue : public InternalPriorityQueue { @@ -44,13 +46,14 @@ class HeapPriorityQueue : public InternalPriorityQueue { queue_.resize(1024); // Initial capacity, can be adjusted } - void add(T toAdd) override { + bool add(T toAdd) override { // Implementation for adding to the priority queue set queue_[size_] = toAdd; size_++; if (size_ >= queue_.size()) { size_ = 0; } + return true; } T poll() override { @@ -79,7 +82,7 @@ class HeapPriorityQueue : public InternalPriorityQueue { return size_ == 0; } - void remove(T toRemove) { + void remove(T toRemove) override { // Implementation for removing an element from the priority queue set auto it = std::find(queue_.begin(), queue_.end(), toRemove); if (it != queue_.end()) { @@ -94,4 +97,60 @@ class HeapPriorityQueue : public InternalPriorityQueue { std::vector queue_; int size_ = 0; }; + +// This class is relevant to flink HeapPriorityQueueSet. +template +class HeapPriorityQueueSet : public HeapPriorityQueue { + +public: + HeapPriorityQueueSet() : HeapPriorityQueue() { + deduplicationMapsByKeyGroup.resize(1); + } + + T poll() override { + T t = HeapPriorityQueue::poll(); + std::unordered_map& deduplicateMap = getDedupMapForElement(t); + auto it = deduplicateMap.find(t); + if (it != deduplicateMap.end()) { + deduplicateMap.erase(it); + } + return t; + } + + bool add(T t) override { + std::unordered_map& deduplicateMap = getDedupMapForElement(t); + if (deduplicateMap.find(t) == deduplicateMap.end()) { + HeapPriorityQueue::add(t); + deduplicateMap.insert({t, t}); + return true; + } else { + return false; + } + } + + void remove(T t) override { + std::unordered_map& deduplicateMap = getDedupMapForElement(t); + auto it = deduplicateMap.find(t); + if (it != deduplicateMap.end()) { + deduplicateMap.erase(it); + HeapPriorityQueue::remove(t); + } + } + + void clear() override { + HeapPriorityQueue::clear(); + for (auto& map : deduplicationMapsByKeyGroup) { + map.clear(); + } + } +private: + mutable std::vector> deduplicationMapsByKeyGroup; + // KeyExtractorFunction keyExtrator; + // KeyGroupRange keyGroupRange + + //TODO: get deDuplicationMap by keyGroupRange + std::unordered_map& getDedupMapForElement(T element) { + return deduplicationMapsByKeyGroup[0]; + } +}; } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/InternalTimerService.h b/velox/experimental/stateful/InternalTimerService.h index 69b8ca0c4f5b..1675e8b394a8 100644 --- a/velox/experimental/stateful/InternalTimerService.h +++ b/velox/experimental/stateful/InternalTimerService.h @@ -42,15 +42,16 @@ class InternalTimerService { void registerProcessingTimeTimer(K key, N ns, int64_t time) { std::shared_ptr> oldHead = processingTimeTimersQueue_.peek(); - processingTimeTimersQueue_.add(std::make_shared>(time, key, ns)); - int64_t nextTriggerTime = oldHead != nullptr ? oldHead->timestamp() : std::numeric_limits::max() ; - if (time < nextTriggerTime) { - if (nextTimer_.has_value()) { - processingTimeService_->cancel(nextTimer_.value()); + if (processingTimeTimersQueue_.add(std::make_shared>(time, key, ns))) { + int64_t nextTriggerTime = oldHead != nullptr ? oldHead->timestamp() : std::numeric_limits::max() ; + if (time < nextTriggerTime) { + if (nextTimer_.has_value()) { + processingTimeService_->cancel(nextTimer_.value()); + } + nextTimer_ = processingTimeService_->registerTimer(time, ProcessingTimerTask(time, [&](int64_t processingTime) { + onProcessingTime(processingTime); + })); } - nextTimer_ = processingTimeService_->registerTimer(time, ProcessingTimerTask(time, [&](int64_t processingTime) { - onProcessingTime(processingTime); - })); } } @@ -120,7 +121,7 @@ class InternalTimerService { std::optional nextTimer_; std::shared_ptr processingTimeService_; HeapPriorityQueue>> eventTimeTimersQueue_; - HeapPriorityQueue>> processingTimeTimersQueue_; + HeapPriorityQueueSet>, HeapTimerHasher, HeapTimerComparator> processingTimeTimersQueue_; }; } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/KeySelector.cpp b/velox/experimental/stateful/KeySelector.cpp index 624dde9a40ae..ce49dcaaefae 100644 --- a/velox/experimental/stateful/KeySelector.cpp +++ b/velox/experimental/stateful/KeySelector.cpp @@ -26,9 +26,9 @@ KeySelector::KeySelector( numPartitions_(numPartitions) { } -std::map KeySelector::partition(const RowVectorPtr& input) { +std::map KeySelector::partition(const RowVectorPtr& input) { if (numPartitions_ == 1) { - return std::map{{0, input}}; + return std::map{{0, input}}; } prepareForInput(input); @@ -39,7 +39,7 @@ std::map KeySelector::partition(const RowVectorPtr& inpu // TODO: this is a optimization, as the RowVector may have be partitioned in // local aggregation, so need not to partition again in global agg, but need // to verify whether the judge condition is enough. - return std::map{{*part, input}}; + return std::map{{*part, input}}; } const auto numInput = input->size(); std::map numOfKeys; @@ -66,7 +66,7 @@ std::map KeySelector::partition(const RowVectorPtr& inpu numOfKeys[partition] = index + 1; } - std::map results; + std::map results; for (auto & [key, partitionSize] : numOfKeys) { auto partitionData = wrapChildren(input, partitionSize, keyToIndexBuffers[key]); results[key] = partitionData; diff --git a/velox/experimental/stateful/KeySelector.h b/velox/experimental/stateful/KeySelector.h index d64a6ea507b3..02f34ca2220f 100644 --- a/velox/experimental/stateful/KeySelector.h +++ b/velox/experimental/stateful/KeySelector.h @@ -22,7 +22,7 @@ namespace facebook::velox::stateful { -/// This class is relevent to flink KeySelector. +/// This class is relevant to flink KeySelector. /// It can partition the RowVector according to the key fields. class KeySelector { public: @@ -31,7 +31,7 @@ class KeySelector { memory::MemoryPool* pool, int numPartitions = 1024); - std::map partition(const RowVectorPtr& input); + std::map partition(const RowVectorPtr& input); private: void prepareForInput(const RowVectorPtr& input); diff --git a/velox/experimental/stateful/LocalWindowAggregator.cpp b/velox/experimental/stateful/LocalWindowAggregator.cpp index eb7bb082ed66..bd64dabfbb14 100644 --- a/velox/experimental/stateful/LocalWindowAggregator.cpp +++ b/velox/experimental/stateful/LocalWindowAggregator.cpp @@ -45,10 +45,9 @@ void LocalWindowAggregator::getOutput() { return; } - std::map keyToData = keySelector_->partition(input_); + std::map keyToData = keySelector_->partition(input_); for (const auto& [key, data] : keyToData) { - std::map sliceEndToData = - sliceAssigner_->partition(data); + std::map sliceEndToData = sliceAssigner_->partition(data); for (const auto& [sliceEnd, data] : sliceEndToData) { // TODO: addElement may have data output. auto windowData = data; diff --git a/velox/experimental/stateful/StreamKeyedOperator.cpp b/velox/experimental/stateful/StreamKeyedOperator.cpp index 0e685c558940..4c6d1e09091a 100644 --- a/velox/experimental/stateful/StreamKeyedOperator.cpp +++ b/velox/experimental/stateful/StreamKeyedOperator.cpp @@ -15,8 +15,6 @@ */ #include "velox/experimental/stateful/StreamKeyedOperator.h" -#include - namespace facebook::velox::stateful { StreamKeyedOperator::StreamKeyedOperator( diff --git a/velox/experimental/stateful/TimerHeapInternalTimer.h b/velox/experimental/stateful/TimerHeapInternalTimer.h index e78d7e3c23c5..f34453760d6b 100644 --- a/velox/experimental/stateful/TimerHeapInternalTimer.h +++ b/velox/experimental/stateful/TimerHeapInternalTimer.h @@ -15,19 +15,22 @@ */ #pragma once +#include "velox/common/base/BitUtil.h" +#include + namespace facebook::velox::stateful { // This class is relevent to flink InternalTimerServiceImpl. template -class TimerHeapInternalTimer { +class TimerHeapInternalTimer { public: - TimerHeapInternalTimer(long timestamp, K key, N ns) + TimerHeapInternalTimer(int64_t timestamp, K key, N ns) : timestamp_(timestamp), key_(key), ns_(ns), keyGroupIndex_(0) {} - long timestamp() { + int64_t timestamp() { return timestamp_; } @@ -49,10 +52,24 @@ class TimerHeapInternalTimer { ns_ == other.ns_; } private: - long timestamp_; + int64_t timestamp_; K key_; N ns_; int keyGroupIndex_; }; +template +struct HeapTimerHasher { + size_t operator() (const std::shared_ptr>& a) const { + return bits::hashMix(a->timestamp(), a->key()); + } +}; + +template +struct HeapTimerComparator { + bool operator() (const std::shared_ptr>& a, const std::shared_ptr>& b) const { + return *a == *b; + } +}; + } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/WatermarkAssigner.cpp b/velox/experimental/stateful/WatermarkAssigner.cpp index cb38df791f4c..a2c8691dc651 100644 --- a/velox/experimental/stateful/WatermarkAssigner.cpp +++ b/velox/experimental/stateful/WatermarkAssigner.cpp @@ -15,8 +15,6 @@ */ #include "velox/experimental/stateful/WatermarkAssigner.h" -#include - namespace facebook::velox::stateful { WatermarkAssigner::WatermarkAssigner( diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index e7796ed9feab..1dd8c596b3c0 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -67,16 +67,14 @@ void WindowAggregator::getOutput() { return; } - std::map keyToData = keySelector_->partition(input_); + std::map keyToData = keySelector_->partition(input_); for (const auto& [key, data] : keyToData) { - std::map sliceEndToData = - sliceAssigner_->assignSliceEnd(data); + std::map sliceEndToData = sliceAssigner_->assignSliceEnd(data); for (const auto& [sliceEnd, data] : sliceEndToData) { auto windowData = data; if (!isEventTime_) { windowTimerService_->registerProcessingTimeTimer(key, sliceEnd, sliceEnd); } - if (isEventTime_ && TimeWindowUtil::isWindowFired(sliceEnd, currentProgress_, shiftTimeZone_)) { // the assigned slice has been triggered, which means current element is late, // but maybe not need to drop @@ -102,7 +100,6 @@ void WindowAggregator::getOutput() { } } } - input_.reset(); } diff --git a/velox/experimental/stateful/window/SliceAssigner.cpp b/velox/experimental/stateful/window/SliceAssigner.cpp index 26db67bfb70d..c0d44068cda2 100644 --- a/velox/experimental/stateful/window/SliceAssigner.cpp +++ b/velox/experimental/stateful/window/SliceAssigner.cpp @@ -36,7 +36,7 @@ SliceAssigner::SliceAssigner( sliceSize_ = std::gcd(size, step); } -std::map SliceAssigner::assignSliceEnd(const RowVectorPtr& input) { +std::map SliceAssigner::assignSliceEnd(const RowVectorPtr& input) { if (rowtimeIndex_ < 0) { // TODO: using Processing Time Service int64_t timestamp_ms = TimeWindowUtil::getCurrentProcessingTime(); diff --git a/velox/experimental/stateful/window/SliceAssigner.h b/velox/experimental/stateful/window/SliceAssigner.h index 13023503a7f9..83ede6f3f16e 100644 --- a/velox/experimental/stateful/window/SliceAssigner.h +++ b/velox/experimental/stateful/window/SliceAssigner.h @@ -31,7 +31,7 @@ class SliceAssigner { WindowType windowType, int rowtimeIndex); - std::map assignSliceEnd(const RowVectorPtr& input); + std::map assignSliceEnd(const RowVectorPtr& input); int64_t getLastWindowEnd(int64_t sliceEnd); diff --git a/velox/experimental/stateful/window/WindowPartitionFunction.cpp b/velox/experimental/stateful/window/WindowPartitionFunction.cpp index a0baceb799de..0b3d5483050a 100644 --- a/velox/experimental/stateful/window/WindowPartitionFunction.cpp +++ b/velox/experimental/stateful/window/WindowPartitionFunction.cpp @@ -15,6 +15,7 @@ */ #include "velox/experimental/stateful/window/WindowPartitionFunction.h" #include "velox/experimental/stateful/window/TimeWindowUtil.h" +#include "velox/experimental/stateful/window/Window.h" #include "velox/vector/FlatVector.h" #include @@ -58,10 +59,10 @@ std::optional WindowPartitionFunction::partition( auto child = input.childAt(rowtimeIndex_); auto ts = child->as>()->valueAt(i); int64_t timestamp = ts.getSeconds() * 1'000 + ts.getNanos() / 1'000'000; - if (windowType_ == 0) { // Hopping window + if (Window::getType(windowType_) == WindowType::HOP) { // Hopping window int64_t start = TimeWindowUtil::getWindowStartWithOffset(timestamp, offset_, sliceSize_); partitions[i] = start + sliceSize_; - } else if (windowType_ == 1) { // Windowed Slice Assigner + } else if (Window::getType(windowType_) == WindowType::TUMBLE) { // Windowed Slice Assigner partitions[i] = timestamp; } else { VELOX_UNSUPPORTED("Unsupported window type: {}", windowType_); From 27903d387843cfd14de25805a1f304b13c789f34 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Thu, 25 Dec 2025 07:11:05 +0000 Subject: [PATCH 12/15] fix111 --- velox/experimental/stateful/InternalPriorityQueue.h | 1 + velox/experimental/stateful/WindowAggregator.cpp | 1 + velox/experimental/stateful/state/StateDescriptor.h | 1 - velox/experimental/stateful/state/StateEntry.h | 2 ++ velox/experimental/stateful/state/StateMap.h | 1 + velox/experimental/stateful/state/StateTable.h | 3 --- velox/experimental/stateful/window/WindowBuffer.cpp | 1 + velox/experimental/stateful/window/WindowBuffer.h | 4 +++- 8 files changed, 9 insertions(+), 5 deletions(-) diff --git a/velox/experimental/stateful/InternalPriorityQueue.h b/velox/experimental/stateful/InternalPriorityQueue.h index 58bbbe6ac48d..7b01374fb0c1 100644 --- a/velox/experimental/stateful/InternalPriorityQueue.h +++ b/velox/experimental/stateful/InternalPriorityQueue.h @@ -142,6 +142,7 @@ class HeapPriorityQueueSet : public HeapPriorityQueue { for (auto& map : deduplicationMapsByKeyGroup) { map.clear(); } + deduplicationMapsByKeyGroup.clear(); } private: mutable std::vector> deduplicationMapsByKeyGroup; diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 1dd8c596b3c0..39f66c5a4b47 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -270,6 +270,7 @@ void WindowAggregator::close() { localAggregator_->close(); } input_.reset(); + windowTimerService_->close(); windowBuffer_->clear(); windowState_->clear(); currentProgress_ = 0; diff --git a/velox/experimental/stateful/state/StateDescriptor.h b/velox/experimental/stateful/state/StateDescriptor.h index a7306e5f7c4b..fe0d2244b642 100644 --- a/velox/experimental/stateful/state/StateDescriptor.h +++ b/velox/experimental/stateful/state/StateDescriptor.h @@ -15,7 +15,6 @@ */ #pragma once -//#include "velox/common/serialization/Serializable.h" #include namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/state/StateEntry.h b/velox/experimental/stateful/state/StateEntry.h index ddd6b310d799..a5186e3015bb 100644 --- a/velox/experimental/stateful/state/StateEntry.h +++ b/velox/experimental/stateful/state/StateEntry.h @@ -15,6 +15,8 @@ */ #pragma once +#include + namespace facebook::velox::stateful { /** diff --git a/velox/experimental/stateful/state/StateMap.h b/velox/experimental/stateful/state/StateMap.h index 3c8e8ddef626..3f9f82866b59 100644 --- a/velox/experimental/stateful/state/StateMap.h +++ b/velox/experimental/stateful/state/StateMap.h @@ -17,6 +17,7 @@ #include "velox/experimental/stateful/state/StateEntry.h" +#include #include namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/state/StateTable.h b/velox/experimental/stateful/state/StateTable.h index a9479034d84e..5a4924f0525b 100644 --- a/velox/experimental/stateful/state/StateTable.h +++ b/velox/experimental/stateful/state/StateTable.h @@ -15,10 +15,7 @@ */ #pragma once -#include "velox/common/base/Exceptions.h" #include "velox/experimental/stateful/state/StateMap.h" -#include -#include #include namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/window/WindowBuffer.cpp b/velox/experimental/stateful/window/WindowBuffer.cpp index 50bbcead02a5..9a348d75f4b1 100644 --- a/velox/experimental/stateful/window/WindowBuffer.cpp +++ b/velox/experimental/stateful/window/WindowBuffer.cpp @@ -23,6 +23,7 @@ namespace facebook::velox::stateful { void RecordsWindowBuffer::addElement(uint32_t key, int64_t sliceEnd, RowVectorPtr& element) { minSliceEnd_ = std::min(sliceEnd, minSliceEnd_); WindowKey windowKey(key, sliceEnd); + std::lock_guard lock(mtx); auto it = buffer_.find(windowKey); if (it != buffer_.end()) { // If the key already exists, we can append the element to the existing list. diff --git a/velox/experimental/stateful/window/WindowBuffer.h b/velox/experimental/stateful/window/WindowBuffer.h index b4a90836a664..f9e59038961b 100644 --- a/velox/experimental/stateful/window/WindowBuffer.h +++ b/velox/experimental/stateful/window/WindowBuffer.h @@ -18,7 +18,7 @@ #include "velox/experimental/stateful/window/WindowKey.h" #include "velox/vector/ComplexVector.h" #include -#include +#include namespace facebook::velox::stateful { @@ -44,6 +44,7 @@ class RecordsWindowBuffer : public WindowBuffer { std::unordered_map>& advanceProgress(int64_t progress) override; void clear() override { + std::lock_guard lock(mtx); buffer_.clear(); minSliceEnd_ = INT64_MAX; } @@ -55,6 +56,7 @@ class RecordsWindowBuffer : public WindowBuffer { std::unordered_map> empty_; int64_t minSliceEnd_ = INT64_MAX; int shiftTimeZone_ = 0; // TODO: support time zone shift + mutable std::mutex mtx; }; } // namespace facebook::velox::stateful From 0130991e36aaa1d72604d02d65e34908a4094a87 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Mon, 29 Dec 2025 03:05:38 +0000 Subject: [PATCH 13/15] xxx111 --- .../stateful/InternalPriorityQueue.h | 267 ++++++++++----- .../stateful/InternalTimerService.h | 24 +- .../stateful/StatefulOperator.cpp | 2 - velox/experimental/stateful/StreamJoin.cpp | 3 - .../stateful/TimerHeapInternalTimer.h | 9 +- velox/experimental/stateful/Triggerable.h | 1 - .../stateful/WindowAggregator.cpp | 5 +- .../stateful/state/HashMapStateBackend.h | 3 +- .../stateful/state/HeapKeyedStateBackend.h | 1 - .../stateful/state/HeapMapState.h | 2 +- .../{StateEntry.h => InternalKeyContext.h} | 43 +-- .../stateful/state/KeyedStateBackend.h | 14 + .../stateful/state/StateDescriptor.h | 2 +- velox/experimental/stateful/state/StateMap.h | 312 ++++++++++++++++-- .../state/StreamOperatorStateHandler.h | 8 +- velox/experimental/stateful/utils/MathUtils.h | 32 ++ velox/experimental/stateful/window/Window.h | 5 + 17 files changed, 572 insertions(+), 161 deletions(-) rename velox/experimental/stateful/state/{StateEntry.h => InternalKeyContext.h} (50%) create mode 100644 velox/experimental/stateful/utils/MathUtils.h diff --git a/velox/experimental/stateful/InternalPriorityQueue.h b/velox/experimental/stateful/InternalPriorityQueue.h index 7b01374fb0c1..baeb4311846f 100644 --- a/velox/experimental/stateful/InternalPriorityQueue.h +++ b/velox/experimental/stateful/InternalPriorityQueue.h @@ -16,7 +16,6 @@ #pragma once #include -#include namespace facebook::velox::stateful { @@ -24,134 +23,234 @@ namespace facebook::velox::stateful { template class InternalPriorityQueue { public: - virtual bool add(T toAdd) = 0; + virtual bool add(const T& toAdd) = 0; + + virtual bool add(T&& toAdd) = 0; virtual T poll() = 0; - virtual T peek() = 0; + virtual T& peek() = 0; virtual void clear() = 0; virtual bool empty() const = 0; - virtual void remove(T toRemove) = 0; + virtual bool remove(const T& toRemove) = 0; + + virtual size_t size() const = 0; + + virtual bool contains(const T& value) const = 0; }; -// This class is relevant to flink HeapPriorityQueue. -// TODO: need to make it equal to flink -template +template < + typename T, + typename Compare = std::less, + typename Hash = std::hash, + typename EqualTo = std::equal_to> class HeapPriorityQueue : public InternalPriorityQueue { public: - HeapPriorityQueue() { - queue_.resize(1024); // Initial capacity, can be adjusted + explicit HeapPriorityQueue(const Compare& comp = Compare()) + : comparator_(comp) {} + + /// Constructs a priority queue with elements from the range [first, last). + template + HeapPriorityQueue(InputIt first, InputIt last, const Compare& comp = Compare()) + : comparator_(comp), heap_(first, last) { + buildHeap(); } - bool add(T toAdd) override { - // Implementation for adding to the priority queue set - queue_[size_] = toAdd; - size_++; - if (size_ >= queue_.size()) { - size_ = 0; - } - return true; + /// Returns the top element without removing it. + /// Throws if the queue is empty. + T& peek() override { + VELOX_CHECK(!empty(), "Cannot peek from an empty priority queue"); + return heap_[0]; } + /// Removes and returns the top element. + /// Throws if the queue is empty. T poll() override { - // Implementation for polling from the priority queue set - size_--; - if (size_ < 0) { - size_ = queue_.size() - 1; + VELOX_CHECK(!empty(), "Cannot poll from an empty priority queu"); + T top = std::move(heap_[0]); + removeAt(0); + return top; + } + + /// Adds an element to the queue. + bool add(const T& value) override { + addImpl(value); + return true; + } + + /// Adds an element to the queue (move version). + bool add(T&& value) override { + addImpl(std::move(value)); + return true; + } + + /// Removes the first occurrence of the specified element from the queue. + /// Returns true if the element was found and removed, false otherwise. + bool remove(const T& value) override { + auto it = valueToIndex_.find(value); + if (it == valueToIndex_.end()) { + return false; } - return queue_[size_]; + removeAt(it->second); + return true; } - T peek() override { - int index = size_ - 1; - if (index < 0) { - index = queue_.size() - 1; + /// Adds all elements from the range [first, last) to the queue. + template + void addAll(InputIt first, InputIt last) { + for (auto it = first; it != last; ++it) { + add(*it); } - return queue_[index]; } - void clear() override { - queue_.clear(); - size_ = 0; + /// Adds all elements from the container to the queue. + template + void addAll(const Container& container) { + addAll(container.begin(), container.end()); } - bool empty() const override { - return size_ == 0; + /// Returns the number of elements in the queue. + size_t size() const override { + return heap_.size(); } - void remove(T toRemove) override { - // Implementation for removing an element from the priority queue set - auto it = std::find(queue_.begin(), queue_.end(), toRemove); - if (it != queue_.end()) { - *it = queue_[size_ - 1]; // Replace with the last element - size_--; - if (size_ < 0) { - size_ = queue_.size() - 1; - } - } + /// Returns true if the queue is empty. + bool empty() const override { + return heap_.empty(); } - private: - std::vector queue_; - int size_ = 0; -}; -// This class is relevant to flink HeapPriorityQueueSet. -template -class HeapPriorityQueueSet : public HeapPriorityQueue { + /// Removes all elements from the queue. + void clear() override { + heap_.clear(); + valueToIndex_.clear(); + } -public: - HeapPriorityQueueSet() : HeapPriorityQueue() { - deduplicationMapsByKeyGroup.resize(1); + /// Returns true if the queue contains the specified element. + bool contains(const T& value) const { + return valueToIndex_.find(value) != valueToIndex_.end(); } - T poll() override { - T t = HeapPriorityQueue::poll(); - std::unordered_map& deduplicateMap = getDedupMapForElement(t); - auto it = deduplicateMap.find(t); - if (it != deduplicateMap.end()) { - deduplicateMap.erase(it); + private: + Compare comparator_; + std::vector heap_; + std::unordered_map valueToIndex_; + + void addImpl(const T& value) { + // Check if value already exists + if (valueToIndex_.find(value) != valueToIndex_.end()) { + // Update existing element + size_t index = valueToIndex_[value]; + heap_[index] = value; + // Re-heapify from this position + percolateUp(index); + percolateDown(index); + } else { + // Add new element + size_t index = heap_.size(); + heap_.push_back(value); + valueToIndex_[value] = index; + percolateUp(index); } - return t; } - bool add(T t) override { - std::unordered_map& deduplicateMap = getDedupMapForElement(t); - if (deduplicateMap.find(t) == deduplicateMap.end()) { - HeapPriorityQueue::add(t); - deduplicateMap.insert({t, t}); - return true; + void addImpl(T&& value) { + // Check if value already exists + auto it = valueToIndex_.find(value); + if (it != valueToIndex_.end()) { + // Update existing element + size_t index = it->second; + heap_[index] = std::move(value); + // Re-heapify from this position + percolateUp(index); + percolateDown(index); } else { - return false; + // Add new element + size_t index = heap_.size(); + heap_.push_back(std::move(value)); + valueToIndex_[heap_.back()] = index; + percolateUp(index); } } - void remove(T t) override { - std::unordered_map& deduplicateMap = getDedupMapForElement(t); - auto it = deduplicateMap.find(t); - if (it != deduplicateMap.end()) { - deduplicateMap.erase(it); - HeapPriorityQueue::remove(t); + void removeAt(size_t index) { + if (index >= heap_.size()) { + return; + } + + // Remove from valueToIndex_ + valueToIndex_.erase(heap_[index]); + + if (index == heap_.size() - 1) { + // Removing the last element + heap_.pop_back(); + return; } + + // Move last element to the removed position + T last = std::move(heap_.back()); + heap_.pop_back(); + heap_[index] = std::move(last); + valueToIndex_[heap_[index]] = index; + + // Re-heapify + percolateUp(index); + percolateDown(index); } - void clear() override { - HeapPriorityQueue::clear(); - for (auto& map : deduplicationMapsByKeyGroup) { - map.clear(); + void percolateUp(size_t index) { + while (index > 0) { + size_t parent = (index - 1) / 2; + if (!comparator_(heap_[index], heap_[parent])) { + break; + } + swapElements(index, parent); + index = parent; + } + } + + void percolateDown(size_t index) { + while (true) { + size_t left = 2 * index + 1; + size_t right = 2 * index + 2; + size_t smallest = index; + + if (left < heap_.size() && comparator_(heap_[left], heap_[smallest])) { + smallest = left; + } + if (right < heap_.size() && comparator_(heap_[right], heap_[smallest])) { + smallest = right; + } + + if (smallest == index) { + break; + } + + swapElements(index, smallest); + index = smallest; } - deduplicationMapsByKeyGroup.clear(); } -private: - mutable std::vector> deduplicationMapsByKeyGroup; - // KeyExtractorFunction keyExtrator; - // KeyGroupRange keyGroupRange - //TODO: get deDuplicationMap by keyGroupRange - std::unordered_map& getDedupMapForElement(T element) { - return deduplicationMapsByKeyGroup[0]; + void swapElements(size_t i, size_t j) { + std::swap(heap_[i], heap_[j]); + valueToIndex_[heap_[i]] = i; + valueToIndex_[heap_[j]] = j; + } + + void buildHeap() { + // Build index map + valueToIndex_.clear(); + for (size_t i = 0; i < heap_.size(); ++i) { + valueToIndex_[heap_[i]] = i; + } + + // Build heap from the bottom up + for (int i = static_cast(heap_.size()) / 2 - 1; i >= 0; --i) { + percolateDown(static_cast(i)); + } } }; + } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/InternalTimerService.h b/velox/experimental/stateful/InternalTimerService.h index 1675e8b394a8..3b4c5f92f1e9 100644 --- a/velox/experimental/stateful/InternalTimerService.h +++ b/velox/experimental/stateful/InternalTimerService.h @@ -21,6 +21,7 @@ #include #include #include +#include #include namespace facebook::velox::stateful { @@ -41,7 +42,7 @@ class InternalTimerService { } void registerProcessingTimeTimer(K key, N ns, int64_t time) { - std::shared_ptr> oldHead = processingTimeTimersQueue_.peek(); + const std::shared_ptr>& oldHead = processingTimeTimersQueue_.peek(); if (processingTimeTimersQueue_.add(std::make_shared>(time, key, ns))) { int64_t nextTriggerTime = oldHead != nullptr ? oldHead->timestamp() : std::numeric_limits::max() ; if (time < nextTriggerTime) { @@ -61,8 +62,9 @@ class InternalTimerService { int64_t currentWatermark() { // TODO: Implement watermark logic if needed. - if (eventTimeTimersQueue_.peek() != nullptr) { - return eventTimeTimersQueue_.peek()->timestamp(); + const std::shared_ptr>& timer = eventTimeTimersQueue_.peek(); + if (timer != nullptr) { + return timer->timestamp(); } return 0; // or some other default value } @@ -73,8 +75,8 @@ class InternalTimerService { } void advanceWatermark(int64_t time) { - while (eventTimeTimersQueue_.peek() != nullptr && - eventTimeTimersQueue_.peek()->timestamp() <= time) { + const std::shared_ptr>& timer = eventTimeTimersQueue_.peek(); + while (timer != nullptr && timer->timestamp() <= time) { auto timer = eventTimeTimersQueue_.poll(); triggerable_->onEventTime(timer); } @@ -101,9 +103,11 @@ class InternalTimerService { triggerOnProcessingTime = false; continue; } - processingTimeTimersQueue_.poll(); - triggerable_->onProcessingTime(timer); - timer = nullptr; + if (!timer) { + processingTimeTimersQueue_.poll(); + triggerable_->onProcessingTime(timer); + timer = nullptr; + } } if (!taskName.empty()) { @@ -120,8 +124,8 @@ class InternalTimerService { Triggerable* triggerable_; std::optional nextTimer_; std::shared_ptr processingTimeService_; - HeapPriorityQueue>> eventTimeTimersQueue_; - HeapPriorityQueueSet>, HeapTimerHasher, HeapTimerComparator> processingTimeTimersQueue_; + HeapPriorityQueue>, HeapTimerComparator, HeapTimerHasher, HeapTimerEquals> eventTimeTimersQueue_; + HeapPriorityQueue>, HeapTimerComparator, HeapTimerHasher, HeapTimerEquals> processingTimeTimersQueue_; }; } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/StatefulOperator.cpp b/velox/experimental/stateful/StatefulOperator.cpp index 4c1990cb43a2..06a332012d4b 100644 --- a/velox/experimental/stateful/StatefulOperator.cpp +++ b/velox/experimental/stateful/StatefulOperator.cpp @@ -17,8 +17,6 @@ #include "velox/experimental/stateful/StatefulTask.h" #include "velox/experimental/stateful/StreamElement.h" -#include - namespace facebook::velox::stateful { void StatefulOperator::initialize() { diff --git a/velox/experimental/stateful/StreamJoin.cpp b/velox/experimental/stateful/StreamJoin.cpp index 15e683752eb8..9345842e1bbc 100644 --- a/velox/experimental/stateful/StreamJoin.cpp +++ b/velox/experimental/stateful/StreamJoin.cpp @@ -15,9 +15,6 @@ */ #include "velox/experimental/stateful/StreamJoin.h" #include "velox/experimental/stateful/join/JoinRecordStateViews.h" -#include "velox/expression/Expr.h" - -#include namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/TimerHeapInternalTimer.h b/velox/experimental/stateful/TimerHeapInternalTimer.h index f34453760d6b..11d99ec812d7 100644 --- a/velox/experimental/stateful/TimerHeapInternalTimer.h +++ b/velox/experimental/stateful/TimerHeapInternalTimer.h @@ -66,10 +66,17 @@ struct HeapTimerHasher { }; template -struct HeapTimerComparator { +struct HeapTimerEquals { bool operator() (const std::shared_ptr>& a, const std::shared_ptr>& b) const { return *a == *b; } }; +template +struct HeapTimerComparator { + bool operator() (const std::shared_ptr>& a, const std::shared_ptr> b) const { + return a->timestamp() < b->timestamp(); + } +}; + } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/Triggerable.h b/velox/experimental/stateful/Triggerable.h index e57b436cb93a..2b6276a7a0ec 100644 --- a/velox/experimental/stateful/Triggerable.h +++ b/velox/experimental/stateful/Triggerable.h @@ -16,7 +16,6 @@ #pragma once #include "velox/experimental/stateful/TimerHeapInternalTimer.h" -#include "velox/vector/ComplexVector.h" #include diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 39f66c5a4b47..2ed5de9d7dc2 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -193,6 +193,7 @@ RowVectorPtr addWindowTimestampToOutput( } void WindowAggregator::onTimer(std::shared_ptr> timer) { + stateHandler()->setCurrentKey(timer->key()); fireWindow(timer->key(), timer->timestamp(), timer->ns()); clearWindow(timer->key(), timer->timestamp(), timer->ns()); } @@ -234,7 +235,7 @@ void WindowAggregator::onEventTime(std::shared_ptr> timer) { - if (timer->timestamp() >= lastTriggeredProcessingTime_) { + if (timer->timestamp() > lastTriggeredProcessingTime_) { lastTriggeredProcessingTime_ = timer->timestamp(); auto windowKeyToData = windowBuffer_->advanceProgress(timer->timestamp()); for (const auto& [windowKey, datas] : windowKeyToData) { @@ -254,8 +255,8 @@ void WindowAggregator::onProcessingTime(std::shared_ptrclear(); - onTimer(timer); } + onTimer(timer); } long WindowAggregator::sliceStateMergeTarget(long sliceToMerge) { diff --git a/velox/experimental/stateful/state/HashMapStateBackend.h b/velox/experimental/stateful/state/HashMapStateBackend.h index 49bbb4d35ddb..dd60b4290c26 100644 --- a/velox/experimental/stateful/state/HashMapStateBackend.h +++ b/velox/experimental/stateful/state/HashMapStateBackend.h @@ -15,13 +15,12 @@ */ #pragma once -#include "velox/common/serialization/Serializable.h" #include "velox/experimental/stateful/state/KeyedStateBackend.h" #include "velox/experimental/stateful/state/StateBackend.h" namespace facebook::velox::stateful { -// This class is relevent to flink org.apache.flink.runtime.state.hashmap.HashMapStateBackend. +// This class is relevant to flink org.apache.flink.runtime.state.hashmap.HashMapStateBackend. class HashMapStateBackend : public StateBackend { public: std::string getName() const { diff --git a/velox/experimental/stateful/state/HeapKeyedStateBackend.h b/velox/experimental/stateful/state/HeapKeyedStateBackend.h index b488bb650fac..b0933f388a17 100644 --- a/velox/experimental/stateful/state/HeapKeyedStateBackend.h +++ b/velox/experimental/stateful/state/HeapKeyedStateBackend.h @@ -15,7 +15,6 @@ */ #pragma once -#include "velox/common/serialization/Serializable.h" #include "velox/experimental/stateful/state/KeyedStateBackend.h" namespace facebook::velox::stateful { diff --git a/velox/experimental/stateful/state/HeapMapState.h b/velox/experimental/stateful/state/HeapMapState.h index b2c5a0982f2d..603741d0eb7a 100644 --- a/velox/experimental/stateful/state/HeapMapState.h +++ b/velox/experimental/stateful/state/HeapMapState.h @@ -20,7 +20,7 @@ namespace facebook::velox::stateful { -// This class is relevent to flink HeapMapState. +// This class is relevant to flink HeapMapState. template class HeapMapState : public MapState { public: diff --git a/velox/experimental/stateful/state/StateEntry.h b/velox/experimental/stateful/state/InternalKeyContext.h similarity index 50% rename from velox/experimental/stateful/state/StateEntry.h rename to velox/experimental/stateful/state/InternalKeyContext.h index a5186e3015bb..69c0c50aac20 100644 --- a/velox/experimental/stateful/state/StateEntry.h +++ b/velox/experimental/stateful/state/InternalKeyContext.h @@ -15,38 +15,21 @@ */ #pragma once -#include - namespace facebook::velox::stateful { -/** - * This class is relevent to flink org.apache.flink.runtime.state.heap.StateMap. - * remove namespace first. It is a simplified implementation. - * @param type of key - * @param type of namespace - * @param type of state - */ -template -class StateEntry { - public: - StateEntry(K key, N ns, S state) - : key_(std::move(key)), namespace_(ns), state_(std::move(state)) {} - - K getKey() { - return key_; - } +template +class InternalKeyContext { +public: + K getCurrentKey() { + return currentKey_; + } - N getNamespace() { - return namespace_; - } + void setCurrentKey(K key) { + currentKey_ = key; + } - S getState() { - return state_; - } - - private: - K key_; - N namespace_; - S state_; +private: + K currentKey_; }; -} // namespace facebook::velox::stateful + +} // namespace facebook::velox::stateful \ No newline at end of file diff --git a/velox/experimental/stateful/state/KeyedStateBackend.h b/velox/experimental/stateful/state/KeyedStateBackend.h index 3baec96ac38f..7a3ed2f5bd4a 100644 --- a/velox/experimental/stateful/state/KeyedStateBackend.h +++ b/velox/experimental/stateful/state/KeyedStateBackend.h @@ -19,9 +19,11 @@ #include "velox/experimental/stateful/state/CheckpointListener.h" #include "velox/experimental/stateful/state/State.h" #include "velox/experimental/stateful/state/StateDescriptor.h" +#include "velox/experimental/stateful/state/InternalKeyContext.h" #include "velox/experimental/stateful/InternalTimerService.h" #include "velox/experimental/stateful/window/Window.h" #include "velox/vector/ComplexVector.h" +#include "velox/common/memory/MemoryPool.h" namespace facebook::velox::stateful { @@ -52,6 +54,18 @@ class KeyedStateBackend : public Snapshotable, public CheckpointListener { virtual std::shared_ptr> createGroupWindowAggTimerService(Triggerable* triggerable) = 0; + + void setCurrentKey(const uint32_t key) { + keyContext_->setCurrentKey(key); + } + + const uint32_t getCurrentKey() { + return keyContext_->getCurrentKey(); + } + +private: + std::shared_ptr> keyContext_; + velox::memory::MemoryPool* memoryPool; }; using KeyedStateBackendPtr = std::shared_ptr; diff --git a/velox/experimental/stateful/state/StateDescriptor.h b/velox/experimental/stateful/state/StateDescriptor.h index fe0d2244b642..25423669e4ca 100644 --- a/velox/experimental/stateful/state/StateDescriptor.h +++ b/velox/experimental/stateful/state/StateDescriptor.h @@ -19,7 +19,7 @@ namespace facebook::velox::stateful { -// This class is relevent to flink org.apache.flink.api.common.StateDescriptor. +// This class is relevant to flink org.apache.flink.api.common.StateDescriptor. class StateDescriptor { public: StateDescriptor(const std::string& name) : name_(name) {} diff --git a/velox/experimental/stateful/state/StateMap.h b/velox/experimental/stateful/state/StateMap.h index 3f9f82866b59..f6e0f366a630 100644 --- a/velox/experimental/stateful/state/StateMap.h +++ b/velox/experimental/stateful/state/StateMap.h @@ -15,15 +15,45 @@ */ #pragma once -#include "velox/experimental/stateful/state/StateEntry.h" +#include "velox/common/base/BitUtil.h" +#include "velox/experimental/stateful/window/Window.h" +#include "velox/experimental/stateful/utils/MathUtils.h" #include #include +#include +#include namespace facebook::velox::stateful { +template +struct StateMapEntry { + + K key_; + N namespace_; + S state_; + uint64_t hash_; + std::shared_ptr> next_; + int32_t entryVersion_; + int32_t stateVersion_; + + StateMapEntry(K key, N ns, S state, uint64_t hash, std::shared_ptr>& next, int32_t entryVersion, int32_t stateVersion) + : key_(key), namespace_(ns), state_(state), hash_(hash), next_(next), entryVersion_(entryVersion), stateVersion_(stateVersion) {} + + StateMapEntry(StateMapEntry other, int entryVersion) + : StateMapEntry(other.key_, other.namespace_, other.state_, other.hash_, other.next_, entryVersion, other.stateVersion_) {} + + bool operator==(const StateMapEntry& entry) const { + return entry.key_ == key_ && entry.namespace_ == namespace_ && entry.state_ == state_; + } + + uint64_t hashCode() { + return bits::hashMix(bits::hashMix(key_, namespace_), state_); + } +}; + /** - * This class is relevent to flink org.apache.flink.runtime.state.heap.StateMap. + * This class is relevant to flink org.apache.flink.runtime.state.heap.StateMap. * remove namespace first. * TODO: It is a simplified implementation, not equal to flink. * @param type of key @@ -32,38 +62,278 @@ namespace facebook::velox::stateful { */ template class StateMap { + +#define MIN_TRANSFERRED_PER_INCREMENTAL_REHASH 4 +#define MINIMUM_CAPACITY 4 +#define MAXIMUM_CAPACITY 1 << 30 +#define DEFAULT_CAPACITY 128 +#define MAX_ARRAY_SIZE std::numeric_limits::max() - 8 + public: - StateMap() { - tempTable_.resize(1024); + StateMap() : StateMap(DEFAULT_CAPACITY) {} + StateMap(int32_t capacity) { + primaryTable_ = empty_; + incrementalRehashTable_ = empty_; + highestRequiredSnapshotVersion_ = 0; + stateMapVersion_ = 0; + primaryTableSize_ = 0; + incrementalRehashTableSize_ = 0; + modCount_ = 0; + + if (capacity < 0) { + threshold_ = -1; + return; + } + + if (capacity < MINIMUM_CAPACITY) { + capacity = MINIMUM_CAPACITY; + } else if (capacity > MAXIMUM_CAPACITY) { + capacity = MAXIMUM_CAPACITY; + } else { + capacity = roundUpToPowerOfTwo(capacity); + } + + primaryTable_ = makeTable(capacity); } S get(K key, N ns) { - auto hash = (std::hash{}(key) + std::hash{}(ns)) % tempTable_.size(); - if (tempTable_[hash] != nullptr - && tempTable_[hash]->getKey() == key - && tempTable_[hash]->getNamespace() == ns) { - return tempTable_[hash]->getState(); + uint64_t hash = computeHashForOperationAndDoIncrementalRehash(key, ns); + int32_t requiredVersion = highestRequiredSnapshotVersion_; + std::vector>>& tab = selectActiveTable(hash); + uint64_t index = hash & (tab.size() - 1); + for (auto e = tab[index]; e != nullptr; e = e->next_) { + if (e->hash_ == hash && e->key_ == key && e->namespace_ == ns) { + if (e->stateVersion_ < requiredVersion) { + if (e->entryVersion_ < requiredVersion) { + e = handleChainedEntryCopyOnWrite(tab, hash & (tab.size() - 1), e); + } + e->stateVersion_ = stateMapVersion_; + /// TODO: use type serializer to copy a state + // e->state_ = + } + return e->state_; + } } - return S(); // Return default state if not found + return nullptr; } void put(K key, N ns, S state) { - // TODO: add enlarge logic. - auto hash = (std::hash{}(key) + std::hash{}(ns)) % tempTable_.size(); - tempTable_[hash] = std::make_shared>(key, ns, state); + std::shared_ptr> e = putEntry(key, ns); + e->state_ = state; + e->stateVersion_ = stateMapVersion_; } - void remove(K key, N ns) { - auto hash = (std::hash{}(key) + std::hash{}(ns)) % tempTable_.size(); - if (tempTable_[hash] != nullptr - && tempTable_[hash]->getKey() == key - && tempTable_[hash]->getNamespace() == ns) { - tempTable_[hash] = nullptr; // Remove the entry + bool containsKey(K key, N ns) { + uint64_t hash = computeHashForOperationAndDoIncrementalRehash(key, ns); + std::vector>>& tab = selectActiveTable(hash); + uint64_t index = hash & (tab.size() - 1); + for (auto e = tab[index]; e != nullptr; e = e->next_) { + if (e->hash_ == hash && e->namespace_ == ns && e->key_ == key) { + return true; + } } + return false; + } + + void remove(K key, N ns) { + removeEntry(key, ns); + } + + size_t size() { + return primaryTableSize_ + incrementalRehashTableSize_; } private: - // TODO: use map temporarily, not equal to flink. - std::vector>> tempTable_; + std::vector>> primaryTable_; + std::vector>> incrementalRehashTable_; + std::vector>> empty_; + + int32_t highestRequiredSnapshotVersion_; + int32_t stateMapVersion_; + int32_t rehashIndex_; + int32_t primaryTableSize_; + int32_t incrementalRehashTableSize_; + uint64_t modCount_; + uint64_t threshold_; + N lastNamespace_; + std::set snapshotVersions_; + + std::shared_ptr> putEntry(K key, N ns) { + uint64_t hash = computeHashForOperationAndDoIncrementalRehash(key, ns); + std::vector>>& tab = selectActiveTable(hash); + uint64_t index = hash & (tab.size() - 1); + + for (auto e = tab[index]; e != nullptr; e = e->next_) { + if (e->hash_ == hash && e->key_ == key && e->namespace_ == ns) { + if (e->entryVersion_ < highestRequiredSnapshotVersion_) { + e = handleChainedEntryCopyOnWrite(tab, index, e); + } + return e; + } + } + ++modCount_; + if (size() > threshold_) { + doubleCapacity(); + } + return addNewStateMapEntry(tab, key,ns, hash); + } + + std::shared_ptr> removeEntry(K key, N ns) { + uint64_t hash = computeHashForOperationAndDoIncrementalRehash(key, ns); + std::vector>>& tab = selectActiveTable(hash); + uint64_t index = hash & (tab.size() - 1); + std::shared_ptr> prev = nullptr; + for (auto e = tab[index]; e != nullptr; prev = e, e = e->next_) { + if (e->hash_ == hash && e->key_ == key && e->namespace_ == ns) { + if (prev == nullptr) { + tab[index] = e->next_; + } else { + if (prev->entryVersion_ < highestRequiredSnapshotVersion_) { + prev = handleChainedEntryCopyOnWrite(tab, index, prev); + } + prev->next_ = e->next_; + } + ++modCount_; + if (tab == primaryTable_) { + --primaryTableSize_; + } else { + --incrementalRehashTableSize_; + } + return e; + } + } + return nullptr; + } + + std::shared_ptr> addNewStateMapEntry(std::vector>>& table, K key, N& ns, uint32_t hash) { + if (ns == lastNamespace_) { + ns = lastNamespace_; + } else { + lastNamespace_ = ns; + } + + uint64_t index = hash & (table.size() - 1); + std::shared_ptr> newEntry = std::make_shared>(key, ns, nullptr, hash, table[index], stateMapVersion_, stateMapVersion_); + table[index] = newEntry; + if (table == primaryTable_) { + ++primaryTableSize_; + } else { + ++incrementalRehashTableSize_; + } + return newEntry; + } + + std::shared_ptr> handleChainedEntryCopyOnWrite( + std::vector>>& tab, uint64_t mapIdx, std::shared_ptr>& untilEntry) { + int32_t required = highestRequiredSnapshotVersion_; + std::shared_ptr> current = tab[mapIdx]; + std::shared_ptr> copy = nullptr; + if (current->entryVersion_ < required) { + copy = std::make_shared>(*current, stateMapVersion_); + tab[mapIdx] = copy; + } else { + copy = current; + } + + while (current != untilEntry) { + current = current->next_; + if (current->entryVersion_ < required) { + copy->next_ = std::make_shared>(*current, stateMapVersion_); + copy = copy->next_; + } else { + copy = current; + } + } + return copy; + } + + std::vector>>& selectActiveTable(uint64_t hashCode) { + return (hashCode & (primaryTable_.size() - 1)) >= rehashIndex_ ? primaryTable_ : incrementalRehashTable_; + } + + uint64_t compositeHash(K key, N ns) { + if constexpr (std::is_same::value) { + return bits::hashMix(key, ns.hashCode()); + } else { + return bits::hashMix(key, ns); + } + } + + void incrementalRehash() { + std::vector>>& oldMap = primaryTable_; + std::vector>>& newMap = incrementalRehashTable_; + + size_t oldCapacity = oldMap.size(); + size_t newMask = newMap.size() - 1; + int32_t requiredVersion = highestRequiredSnapshotVersion_; + int32_t rhIdx = rehashIndex_; + int32_t transferred = 0; + + while (transferred < MIN_TRANSFERRED_PER_INCREMENTAL_REHASH) { + std::shared_ptr> e = oldMap[rhIdx]; + while (e != nullptr) { + if (e->entryVersion_ < requiredVersion) { + e = std::make_shared>(*e, stateMapVersion_); + } + std::shared_ptr> n = e->next_; + uint64_t pos = e->hash_ & newMask; + e->next_ = newMap[pos]; + newMap[pos] = e; + e = n; + ++transferred; + } + + oldMap[rhIdx] = nullptr; + if (++rhIdx == oldCapacity) { + primaryTable_ = newMap; + incrementalRehashTable_ = empty_; + primaryTableSize_ += incrementalRehashTableSize_; + incrementalRehashTableSize_ = 0; + rehashIndex_ = 0; + return ; + } + } + + primaryTableSize_ -= transferred; + incrementalRehashTableSize_ += transferred; + rehashIndex_ = rhIdx; + } + + bool isRehashing() { + return empty_ != incrementalRehashTable_; + } + + uint64_t computeHashForOperationAndDoIncrementalRehash(K key, N ns) { + if (isRehashing()) { + incrementalRehash(); + } + return compositeHash(key, ns); + } + + void doubleCapacity() { + VELOX_CHECK(!isRehashing(), "There is already a rehash in progress."); + std::vector>>& oldMap = primaryTable_; + int oldCapacity = oldMap.size(); + if (oldCapacity == MAXIMUM_CAPACITY) { + return; + } + incrementalRehashTable_ = makeTable(oldCapacity * 2); + } + + std::vector>> makeTable(uint64_t newCapacity) { + if (newCapacity < MAXIMUM_CAPACITY) { + threshold_ = (newCapacity >> 1) + (newCapacity >> 2); + } else { + if (size() > MAX_ARRAY_SIZE) { + VELOX_FAIL("Maximum capacity of CopyOnWriteStateMap is reached and the job cannot continue."); + } else { + LOG(WARNING) << "Maximum capacity of 2^30 in StateMap reached. Cannot increase hash map size."; + threshold_ = MAX_ARRAY_SIZE; + } + } + std::vector>> newMap; + newMap.resize(newCapacity); + return newMap; + } }; } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/state/StreamOperatorStateHandler.h b/velox/experimental/stateful/state/StreamOperatorStateHandler.h index 67eb2165402f..62fff9660af3 100644 --- a/velox/experimental/stateful/state/StreamOperatorStateHandler.h +++ b/velox/experimental/stateful/state/StreamOperatorStateHandler.h @@ -85,8 +85,12 @@ class StreamOperatorStateHandler { // TODO: should make it using template std::shared_ptr> createGroupWindowAggTimerService( Triggerable* triggerable) { - return keyedStateBackend_->createGroupWindowAggTimerService(triggerable); -} + return keyedStateBackend_->createGroupWindowAggTimerService(triggerable); + } + + void setCurrentKey(uint32_t key) { + keyedStateBackend_->setCurrentKey(key); + } private: int operatorId_; diff --git a/velox/experimental/stateful/utils/MathUtils.h b/velox/experimental/stateful/utils/MathUtils.h new file mode 100644 index 000000000000..1f6db6989fa7 --- /dev/null +++ b/velox/experimental/stateful/utils/MathUtils.h @@ -0,0 +1,32 @@ +/* + * Copyright (c) Facebook, Inc. and its affiliates. + * + * Licensed 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. + */ +#pragma once + +#include + +namespace facebook::velox::stateful { + + static int roundUpToPowerOfTwo(int32_t x) { + x = x - 1; + x |= x >> 1; + x |= x >> 2; + x |= x >> 4; + x |= x >> 8; + x |= x >> 16; + return x + 1; + } + +} \ No newline at end of file diff --git a/velox/experimental/stateful/window/Window.h b/velox/experimental/stateful/window/Window.h index 3672a81940d8..6f96e8060e87 100644 --- a/velox/experimental/stateful/window/Window.h +++ b/velox/experimental/stateful/window/Window.h @@ -15,6 +15,7 @@ */ #pragma once +#include "velox/common/base/BitUtil.h" #include "velox/common/base/Exceptions.h" #include #include @@ -94,6 +95,10 @@ class TimeWindow : public Window { return start_ >= 0 && end_ >= start_; } + uint64_t hashCode() { + return bits::hashMix(end_, start_); + } + private: int64_t start_; int64_t end_; From 3f4e7a77e5101ab243180643c9ecdfb7f522318e Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Mon, 29 Dec 2025 12:10:00 +0000 Subject: [PATCH 14/15] fix1111 --- .../experimental/stateful/GroupWindowAggregator.h | 1 - .../experimental/stateful/InternalPriorityQueue.h | 15 +++++++-------- .../experimental/stateful/InternalTimerService.h | 14 ++++++-------- velox/experimental/stateful/WindowAggregator.cpp | 1 - velox/experimental/stateful/state/StateMap.h | 3 +-- .../stateful/window/WindowPartitionFunction.cpp | 1 - .../stateful/window/WindowProcessFunction.h | 1 - 7 files changed, 14 insertions(+), 22 deletions(-) diff --git a/velox/experimental/stateful/GroupWindowAggregator.h b/velox/experimental/stateful/GroupWindowAggregator.h index a262a7708d69..f0e4c8010378 100644 --- a/velox/experimental/stateful/GroupWindowAggregator.h +++ b/velox/experimental/stateful/GroupWindowAggregator.h @@ -21,7 +21,6 @@ #include "velox/experimental/stateful/TimerHeapInternalTimer.h" #include "velox/experimental/stateful/Triggerable.h" #include "velox/experimental/stateful/window/SliceAssigner.h" -#include "velox/experimental/stateful/window/WindowBuffer.h" #include "velox/experimental/stateful/window/WindowProcessFunction.h" #include "velox/experimental/stateful/window/WindowTrigger.h" diff --git a/velox/experimental/stateful/InternalPriorityQueue.h b/velox/experimental/stateful/InternalPriorityQueue.h index baeb4311846f..39bc91a32a9e 100644 --- a/velox/experimental/stateful/InternalPriorityQueue.h +++ b/velox/experimental/stateful/InternalPriorityQueue.h @@ -70,9 +70,7 @@ class HeapPriorityQueue : public InternalPriorityQueue { /// Throws if the queue is empty. T poll() override { VELOX_CHECK(!empty(), "Cannot poll from an empty priority queu"); - T top = std::move(heap_[0]); - removeAt(0); - return top; + return removeAt(0); } /// Adds an element to the queue. @@ -175,18 +173,18 @@ class HeapPriorityQueue : public InternalPriorityQueue { } } - void removeAt(size_t index) { + T removeAt(size_t index) { if (index >= heap_.size()) { - return; + VELOX_FAIL("Logical error, the removed index {} is greater than heap size {}", index, heap_.size()); } - + T t = std::move(heap_[index]); // Remove from valueToIndex_ - valueToIndex_.erase(heap_[index]); + valueToIndex_.erase(t); if (index == heap_.size() - 1) { // Removing the last element heap_.pop_back(); - return; + return t; } // Move last element to the removed position @@ -198,6 +196,7 @@ class HeapPriorityQueue : public InternalPriorityQueue { // Re-heapify percolateUp(index); percolateDown(index); + return t; } void percolateUp(size_t index) { diff --git a/velox/experimental/stateful/InternalTimerService.h b/velox/experimental/stateful/InternalTimerService.h index 3b4c5f92f1e9..c4da00b23ca4 100644 --- a/velox/experimental/stateful/InternalTimerService.h +++ b/velox/experimental/stateful/InternalTimerService.h @@ -42,7 +42,7 @@ class InternalTimerService { } void registerProcessingTimeTimer(K key, N ns, int64_t time) { - const std::shared_ptr>& oldHead = processingTimeTimersQueue_.peek(); + const std::shared_ptr>& oldHead = processingTimeTimersQueue_.empty() ? nullptr : processingTimeTimersQueue_.peek(); if (processingTimeTimersQueue_.add(std::make_shared>(time, key, ns))) { int64_t nextTriggerTime = oldHead != nullptr ? oldHead->timestamp() : std::numeric_limits::max() ; if (time < nextTriggerTime) { @@ -62,7 +62,7 @@ class InternalTimerService { int64_t currentWatermark() { // TODO: Implement watermark logic if needed. - const std::shared_ptr>& timer = eventTimeTimersQueue_.peek(); + const std::shared_ptr>& timer = eventTimeTimersQueue_.empty() ? nullptr : eventTimeTimersQueue_.peek(); if (timer != nullptr) { return timer->timestamp(); } @@ -75,7 +75,7 @@ class InternalTimerService { } void advanceWatermark(int64_t time) { - const std::shared_ptr>& timer = eventTimeTimersQueue_.peek(); + const std::shared_ptr>& timer = eventTimeTimersQueue_.empty() ? nullptr : eventTimeTimersQueue_.peek(); while (timer != nullptr && timer->timestamp() <= time) { auto timer = eventTimeTimersQueue_.poll(); triggerable_->onEventTime(timer); @@ -103,11 +103,9 @@ class InternalTimerService { triggerOnProcessingTime = false; continue; } - if (!timer) { - processingTimeTimersQueue_.poll(); - triggerable_->onProcessingTime(timer); - timer = nullptr; - } + processingTimeTimersQueue_.poll(); + triggerable_->onProcessingTime(timer); + timer = nullptr; } if (!taskName.empty()) { diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 2ed5de9d7dc2..8d9972eebd73 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -193,7 +193,6 @@ RowVectorPtr addWindowTimestampToOutput( } void WindowAggregator::onTimer(std::shared_ptr> timer) { - stateHandler()->setCurrentKey(timer->key()); fireWindow(timer->key(), timer->timestamp(), timer->ns()); clearWindow(timer->key(), timer->timestamp(), timer->ns()); } diff --git a/velox/experimental/stateful/state/StateMap.h b/velox/experimental/stateful/state/StateMap.h index f6e0f366a630..b570bad5c49a 100644 --- a/velox/experimental/stateful/state/StateMap.h +++ b/velox/experimental/stateful/state/StateMap.h @@ -162,7 +162,6 @@ class StateMap { uint64_t hash = computeHashForOperationAndDoIncrementalRehash(key, ns); std::vector>>& tab = selectActiveTable(hash); uint64_t index = hash & (tab.size() - 1); - for (auto e = tab[index]; e != nullptr; e = e->next_) { if (e->hash_ == hash && e->key_ == key && e->namespace_ == ns) { if (e->entryVersion_ < highestRequiredSnapshotVersion_) { @@ -205,7 +204,7 @@ class StateMap { return nullptr; } - std::shared_ptr> addNewStateMapEntry(std::vector>>& table, K key, N& ns, uint32_t hash) { + std::shared_ptr> addNewStateMapEntry(std::vector>>& table, K key, N& ns, uint64_t hash) { if (ns == lastNamespace_) { ns = lastNamespace_; } else { diff --git a/velox/experimental/stateful/window/WindowPartitionFunction.cpp b/velox/experimental/stateful/window/WindowPartitionFunction.cpp index 0b3d5483050a..75ee06fa4593 100644 --- a/velox/experimental/stateful/window/WindowPartitionFunction.cpp +++ b/velox/experimental/stateful/window/WindowPartitionFunction.cpp @@ -16,7 +16,6 @@ #include "velox/experimental/stateful/window/WindowPartitionFunction.h" #include "velox/experimental/stateful/window/TimeWindowUtil.h" #include "velox/experimental/stateful/window/Window.h" -#include "velox/vector/FlatVector.h" #include diff --git a/velox/experimental/stateful/window/WindowProcessFunction.h b/velox/experimental/stateful/window/WindowProcessFunction.h index abe455851076..1789c3c47302 100644 --- a/velox/experimental/stateful/window/WindowProcessFunction.h +++ b/velox/experimental/stateful/window/WindowProcessFunction.h @@ -23,7 +23,6 @@ #include "velox/vector/ComplexVector.h" #include -#include #include namespace facebook::velox::stateful { From d630c854f910ceaf6e937c078ddc75501b9746e6 Mon Sep 17 00:00:00 2001 From: zouyunhe Date: Mon, 5 Jan 2026 01:55:16 +0000 Subject: [PATCH 15/15] fix 111 --- .../stateful/InternalTimerService.h | 27 ++++++++++--------- velox/experimental/stateful/Triggerable.h | 9 +++++++ .../stateful/WindowAggregator.cpp | 8 ++++-- .../experimental/stateful/WindowAggregator.h | 1 + velox/experimental/stateful/state/StateMap.h | 8 +++--- .../stateful/window/SliceAssigner.cpp | 10 +++---- .../stateful/window/WindowBuffer.cpp | 1 - .../stateful/window/WindowBuffer.h | 2 -- 8 files changed, 38 insertions(+), 28 deletions(-) diff --git a/velox/experimental/stateful/InternalTimerService.h b/velox/experimental/stateful/InternalTimerService.h index c4da00b23ca4..33f04cebe2ae 100644 --- a/velox/experimental/stateful/InternalTimerService.h +++ b/velox/experimental/stateful/InternalTimerService.h @@ -97,19 +97,22 @@ class InternalTimerService { nextTimer_ = std::nullopt; std::shared_ptr> timer = nullptr; bool triggerOnProcessingTime = true; - while (triggerOnProcessingTime && !processingTimeTimersQueue_.empty()) { - timer = processingTimeTimersQueue_.peek(); - if (!timer || timer->timestamp() > time) { - triggerOnProcessingTime = false; - continue; + const std::shared_ptr mtx = triggerable_->getMutex(); + if (mtx) { + std::lock_guard lock(*mtx); + while (triggerOnProcessingTime && !processingTimeTimersQueue_.empty()) { + timer = processingTimeTimersQueue_.peek(); + if (!timer || timer->timestamp() > time) { + triggerOnProcessingTime = false; + continue; + } + processingTimeTimersQueue_.poll(); + triggerable_->onProcessingTime(timer); + timer = nullptr; + } + if (!taskName.empty()) { + processingTimeService_->unregister(taskName); } - processingTimeTimersQueue_.poll(); - triggerable_->onProcessingTime(timer); - timer = nullptr; - } - - if (!taskName.empty()) { - processingTimeService_->unregister(taskName); } if (timer != nullptr && !nextTimer_.has_value()) { diff --git a/velox/experimental/stateful/Triggerable.h b/velox/experimental/stateful/Triggerable.h index 2b6276a7a0ec..427d29f8c354 100644 --- a/velox/experimental/stateful/Triggerable.h +++ b/velox/experimental/stateful/Triggerable.h @@ -18,6 +18,7 @@ #include "velox/experimental/stateful/TimerHeapInternalTimer.h" #include +#include namespace facebook::velox::stateful { @@ -25,11 +26,19 @@ namespace facebook::velox::stateful { template class Triggerable { public: + Triggerable() { + mtx_ = std::make_shared(); + } virtual void onEventTime( std::shared_ptr> timer) = 0; virtual void onProcessingTime( std::shared_ptr> timer) {} + + const std::shared_ptr getMutex() { return mtx_; } + +protected: + std::shared_ptr mtx_; }; } // namespace facebook::velox::stateful diff --git a/velox/experimental/stateful/WindowAggregator.cpp b/velox/experimental/stateful/WindowAggregator.cpp index 8d9972eebd73..d05b2e4f809e 100644 --- a/velox/experimental/stateful/WindowAggregator.cpp +++ b/velox/experimental/stateful/WindowAggregator.cpp @@ -15,6 +15,7 @@ */ #include #include +#include #include "velox/type/Type.h" #include "velox/vector/ComplexVector.h" #include "velox/experimental/stateful/WindowAggregator.h" @@ -34,7 +35,7 @@ WindowAggregator::WindowAggregator( const bool isEventTime, const int windowStartIndex, const int windowEndIndex) - : StatefulOperator(std::move(globalAggregator), std::move(targets)), + : StatefulOperator(std::move(globalAggregator), std::move(targets)), Triggerable(), localAggregator_(std::move(localAggregator)), keySelector_(std::move(keySelector)), sliceAssigner_(std::move(sliceAssigner)), @@ -96,6 +97,7 @@ void WindowAggregator::getOutput() { } } else { // the assigned slice hasn't been triggered, accumulate into the assigned slice + std::lock_guard lock(*mtx_); windowBuffer_->addElement(key, sliceEnd, windowData); } } @@ -253,7 +255,9 @@ void WindowAggregator::onProcessingTime(std::shared_ptrupdate(windowKey.key(), windowKey.window(), newAcc); } } - windowBuffer_->clear(); + if (!windowKeyToData.empty()) { + windowBuffer_->clear(); + } } onTimer(timer); } diff --git a/velox/experimental/stateful/WindowAggregator.h b/velox/experimental/stateful/WindowAggregator.h index cf9945b3cfa6..eb364bcdd840 100644 --- a/velox/experimental/stateful/WindowAggregator.h +++ b/velox/experimental/stateful/WindowAggregator.h @@ -16,6 +16,7 @@ #pragma once #include +#include #include "velox/exec/Driver.h" #include "velox/exec/Operator.h" #include "velox/experimental/stateful/InternalTimerService.h" diff --git a/velox/experimental/stateful/state/StateMap.h b/velox/experimental/stateful/state/StateMap.h index b570bad5c49a..92962126fb40 100644 --- a/velox/experimental/stateful/state/StateMap.h +++ b/velox/experimental/stateful/state/StateMap.h @@ -147,12 +147,11 @@ class StateMap { std::vector>> primaryTable_; std::vector>> incrementalRehashTable_; std::vector>> empty_; - int32_t highestRequiredSnapshotVersion_; int32_t stateMapVersion_; int32_t rehashIndex_; - int32_t primaryTableSize_; - int32_t incrementalRehashTableSize_; + uint64_t primaryTableSize_; + uint64_t incrementalRehashTableSize_; uint64_t modCount_; uint64_t threshold_; N lastNamespace_; @@ -181,8 +180,7 @@ class StateMap { uint64_t hash = computeHashForOperationAndDoIncrementalRehash(key, ns); std::vector>>& tab = selectActiveTable(hash); uint64_t index = hash & (tab.size() - 1); - std::shared_ptr> prev = nullptr; - for (auto e = tab[index]; e != nullptr; prev = e, e = e->next_) { + for (std::shared_ptr> e = tab[index], prev = nullptr; e != nullptr; prev = e, e = e->next_) { if (e->hash_ == hash && e->key_ == key && e->namespace_ == ns) { if (prev == nullptr) { tab[index] = e->next_; diff --git a/velox/experimental/stateful/window/SliceAssigner.cpp b/velox/experimental/stateful/window/SliceAssigner.cpp index c0d44068cda2..418f604adad1 100644 --- a/velox/experimental/stateful/window/SliceAssigner.cpp +++ b/velox/experimental/stateful/window/SliceAssigner.cpp @@ -38,15 +38,13 @@ SliceAssigner::SliceAssigner( std::map SliceAssigner::assignSliceEnd(const RowVectorPtr& input) { if (rowtimeIndex_ < 0) { - // TODO: using Processing Time Service - int64_t timestamp_ms = TimeWindowUtil::getCurrentProcessingTime(); - if (windowType_ == WindowType::TUMBLE) { // tumble window - // TODO:: support get utcTimestamp by timezone. - int64_t utcTimestamp = TimeWindowUtil::toEpochMillsForTimer(timestamp_ms, 0); + int64_t timestampMs = TimeWindowUtil::getCurrentProcessingTime(); + if (windowType_ == WindowType::TUMBLE) { + int64_t utcTimestamp = TimeWindowUtil::toEpochMillsForTimer(timestampMs, 0); int64_t windowStart = stateful::TimeWindowUtil::getWindowStartWithOffset(utcTimestamp, offset_, size_); return {{windowStart + size_, input}}; } else { - return {{timestamp_ms, input}}; + return {{timestampMs, input}}; } } return keySelector_->partition(input); diff --git a/velox/experimental/stateful/window/WindowBuffer.cpp b/velox/experimental/stateful/window/WindowBuffer.cpp index 9a348d75f4b1..50bbcead02a5 100644 --- a/velox/experimental/stateful/window/WindowBuffer.cpp +++ b/velox/experimental/stateful/window/WindowBuffer.cpp @@ -23,7 +23,6 @@ namespace facebook::velox::stateful { void RecordsWindowBuffer::addElement(uint32_t key, int64_t sliceEnd, RowVectorPtr& element) { minSliceEnd_ = std::min(sliceEnd, minSliceEnd_); WindowKey windowKey(key, sliceEnd); - std::lock_guard lock(mtx); auto it = buffer_.find(windowKey); if (it != buffer_.end()) { // If the key already exists, we can append the element to the existing list. diff --git a/velox/experimental/stateful/window/WindowBuffer.h b/velox/experimental/stateful/window/WindowBuffer.h index f9e59038961b..cfe0bd6f61e5 100644 --- a/velox/experimental/stateful/window/WindowBuffer.h +++ b/velox/experimental/stateful/window/WindowBuffer.h @@ -44,7 +44,6 @@ class RecordsWindowBuffer : public WindowBuffer { std::unordered_map>& advanceProgress(int64_t progress) override; void clear() override { - std::lock_guard lock(mtx); buffer_.clear(); minSliceEnd_ = INT64_MAX; } @@ -56,7 +55,6 @@ class RecordsWindowBuffer : public WindowBuffer { std::unordered_map> empty_; int64_t minSliceEnd_ = INT64_MAX; int shiftTimeZone_ = 0; // TODO: support time zone shift - mutable std::mutex mtx; }; } // namespace facebook::velox::stateful