-
Notifications
You must be signed in to change notification settings - Fork 457
feat(interactive): Impl kafka wal writer and wal paresr #4518
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
zhanglei1949
wants to merge
18
commits into
alibaba:main
Choose a base branch
from
zhanglei1949:impl-kafka-writer
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from 13 commits
Commits
Show all changes
18 commits
Select commit
Hold shift + click to select a range
62976bc
fix(interactive): disable repair wheel in coordinator dockerfile (#4517)
siyuan0322 56d4fc5
fix cmakelist.txt
liulx20 cf83036
Merge branch 'main' into impl-kafka-writer
liulx20 6f32349
fix ci
liulx20 3b11fe1
get parameters from uri
liulx20 46a5f51
refactor ci
liulx20 3182360
fix offset
liulx20 81cb823
insert more vertices
liulx20 0d7bd50
fixing ci
zhanglei1949 be2b581
fixing ci
zhanglei1949 222a49e
submodule
zhanglei1949 f32e3ea
append wal when abort
liulx20 90b5892
refactor ingest wal
liulx20 cc212e7
Update docs/flex/interactive/development/dev_and_test.md
zhanglei1949 2dfd958
Update docs/flex/interactive/development/dev_and_test.md
zhanglei1949 a3e3338
fix
liulx20 c5b4931
fix ci
liulx20 2f15484
fix
liulx20 File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -367,6 +367,28 @@ jobs: | |||||
export ENGINE_TYPE=hiactor | ||||||
bash hqps_adhoc_test.sh ${INTERACTIVE_WORKSPACE} graph_algo RBO | ||||||
|
||||||
- name: Run kafka wal test | ||||||
env: | ||||||
GS_TEST_DIR: ${{ github.workspace }}/gstest | ||||||
run: | | ||||||
git submodule update --init | ||||||
cd ${GITHUB_WORKSPACE}/flex/build | ||||||
cmake .. -DBUILD_KAFKA_WAL_WRITER_PARSER=ON && make -j$(nproc) | ||||||
wget https://dlcdn.apache.org/kafka/3.9.0/kafka_2.13-3.9.0.tgz | ||||||
tar -zxf kafka_2.13-3.9.0.tgz | ||||||
cd kafka_2.13-3.9.0 | ||||||
KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)" | ||||||
bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/kraft/reconfig-server.properties | ||||||
bin/kafka-server-start.sh config/kraft/reconfig-server.properties & | ||||||
|
||||||
bin/kafka-topics.sh --create --topic kafka-test --bootstrap-server localhost:9092 | ||||||
../tests/hqps/kafka_test localhost:9092 kafka-test | ||||||
bin/kafka-topics.sh --delete --topic kafka-test --bootstrap-server localhost:9092 | ||||||
bin/kafka-topics.sh --create --topic kafka-test --bootstrap-server localhost:9092 | ||||||
../tests/hqps/kafka_wal_ingester_test .. localhost:9092 kafka-test | ||||||
bin/kafka-topics.sh --delete --topic kafka-test --bootstrap-server localhost:9092 | ||||||
ps aux | grep kafka | grep -v grep | awk '{print $2}' | xargs kill -9 | ||||||
|
||||||
Comment on lines
+390
to
+391
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The process termination command may kill unrelated processes. Consider refining the filter to specifically target the Kafka process started by the workflow.
Suggested change
Copilot uses AI. Check for mistakes. Positive FeedbackNegative Feedback |
||||||
- name: Run Gremlin test on modern graph | ||||||
env: | ||||||
GS_TEST_DIR: ${{ github.workspace }}/gstest | ||||||
|
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,205 @@ | ||
/** Copyright 2020 Alibaba Group Holding Limited. | ||
* | ||
* 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 "flex/engines/graph_db/app/kafka_wal_ingester_app.h" | ||
#include "cppkafka/cppkafka.h" | ||
#include "flex/engines/graph_db/database/graph_db.h" | ||
#include "flex/engines/graph_db/database/wal/kafka_wal_parser.h" | ||
|
||
namespace gs { | ||
#ifdef BUILD_KAFKA_WAL_WRITER_PARSER | ||
|
||
struct WalIngester { | ||
GraphDBSession& session_; | ||
timestamp_t begin_; | ||
timestamp_t end_; | ||
timestamp_t ingested_plus_one_; | ||
std::vector<std::string> data_; | ||
// 0: not exist, 1: exist, 2: ingested | ||
std::vector<uint8_t> states_; | ||
|
||
void resize() { | ||
size_t n = data_.size(); | ||
std::vector<std::string> new_data(n + 4096); | ||
std::vector<uint8_t> new_states(n + 4096, 0); | ||
size_t idx = (ingested_plus_one_ - begin_) % n; | ||
for (size_t i = 0; i < n; ++i) { | ||
new_data[i] = data_[idx]; | ||
new_states[i] = states_[idx]; | ||
if (states_[idx]) { | ||
end_ = ingested_plus_one_ + i + 1; | ||
} | ||
++idx; | ||
idx %= n; | ||
} | ||
data_ = std::move(new_data); | ||
states_ = std::move(new_states); | ||
begin_ = ingested_plus_one_; | ||
} | ||
WalIngester(GraphDBSession& session, timestamp_t cur) | ||
: session_(session), | ||
begin_(cur), | ||
end_(cur), | ||
ingested_plus_one_(cur), | ||
data_(4096), | ||
states_(4096, 0) {} | ||
|
||
bool empty() const { return ingested_plus_one_ == end_; } | ||
|
||
void ingest_impl(const std::string& data) { | ||
auto header = reinterpret_cast<const WalHeader*>(data.data()); | ||
if (header->type == 0) { | ||
InsertTransaction::IngestWal( | ||
session_.graph(), header->timestamp, | ||
const_cast<char*>(data.data()) + sizeof(WalHeader), header->length, | ||
session_.allocator()); | ||
} else { | ||
auto txn = session_.GetUpdateTransaction(); | ||
auto header = reinterpret_cast<const WalHeader*>(data.data()); | ||
txn.IngestWal(session_.graph(), session_.db().work_dir(), | ||
header->timestamp, | ||
const_cast<char*>(data.data()) + sizeof(WalHeader), | ||
header->length, session_.allocator()); | ||
txn.Commit(); | ||
} | ||
} | ||
|
||
void ingest() { | ||
size_t idx = (ingested_plus_one_ - begin_) % data_.size(); | ||
bool flag = false; | ||
while (states_[idx] == 2 || states_[idx] == 1) { | ||
if (states_[idx] == 1) { | ||
ingest_impl(data_[idx]); | ||
} | ||
states_[idx] = 0; | ||
++ingested_plus_one_; | ||
++idx; | ||
idx %= data_.size(); | ||
flag = true; | ||
} | ||
if (flag) { | ||
session_.commit(ingested_plus_one_); | ||
} | ||
} | ||
void push(const std::string& data) { | ||
auto header = reinterpret_cast<const WalHeader*>(data.data()); | ||
if (header->timestamp < begin_) { | ||
LOG(ERROR) << "Invalid timestamp: " << header->timestamp; | ||
return; | ||
} | ||
size_t index; | ||
size_t n = data_.size(); | ||
if (header->timestamp < end_) { | ||
index = (header->timestamp - begin_) % n; | ||
} else if (header->timestamp - begin_ < n) { | ||
index = header->timestamp - begin_; | ||
end_ = header->timestamp + 1; | ||
} else { | ||
ingest(); | ||
while (header->timestamp - ingested_plus_one_ + 1 > states_.size()) { | ||
resize(); | ||
} | ||
index = (header->timestamp - begin_) % states_.size(); | ||
end_ = header->timestamp + 1; | ||
} | ||
if (header->length == 0) { | ||
states_[index] = 2; | ||
} else if (header->type == 0) { | ||
ingest_impl(data); | ||
states_[index] = 2; | ||
} else { | ||
states_[index] = 1; | ||
data_[index] = data; | ||
} | ||
} | ||
}; | ||
|
||
class KafkaWalConsumer { | ||
public: | ||
static constexpr const std::chrono::milliseconds POLL_TIMEOUT = | ||
std::chrono::milliseconds(100); | ||
|
||
// always track all partitions and from begining | ||
KafkaWalConsumer(WalIngester& ingester, cppkafka::Configuration config, | ||
const std::string& topic_name, int32_t thread_num) | ||
: ingester_(ingester) { | ||
auto topic_partitions = get_all_topic_partitions(config, topic_name, false); | ||
consumers_.reserve(topic_partitions.size()); | ||
for (size_t i = 0; i < topic_partitions.size(); ++i) { | ||
consumers_.emplace_back(std::make_unique<cppkafka::Consumer>(config)); | ||
consumers_.back()->assign({topic_partitions[i]}); | ||
} | ||
} | ||
|
||
void poll() { | ||
for (auto& consumer : consumers_) { | ||
auto msgs = consumer->poll_batch(1024); | ||
for (const auto& msg : msgs) { | ||
if (msg) { | ||
if (msg.get_error()) { | ||
if (!msg.is_eof()) { | ||
LOG(INFO) << "[+] Received error notification: " | ||
<< msg.get_error(); | ||
} | ||
} else { | ||
std::string payload = msg.get_payload(); | ||
ingester_.push(payload); | ||
consumer->commit(msg); | ||
} | ||
} | ||
} | ||
} | ||
} | ||
|
||
private: | ||
std::vector<std::unique_ptr<cppkafka::Consumer>> consumers_; | ||
WalIngester& ingester_; | ||
}; | ||
|
||
void Ingest(const std::string& data, GraphDBSession& session) {} | ||
|
||
bool KafkaWalIngesterApp::Query(GraphDBSession& graph, Decoder& input, | ||
Encoder& output) { | ||
cppkafka::Configuration config; | ||
std::string topic_name; | ||
while (!input.empty()) { | ||
auto key = input.get_string(); | ||
auto value = input.get_string(); | ||
if (key == "topic_name") { | ||
topic_name = value; | ||
} else { | ||
config.set(std::string(key), std::string(value)); | ||
} | ||
} | ||
LOG(INFO) << "Kafka brokers: " << config.get("metadata.broker.list"); | ||
timestamp_t cur_ts = graph.db().get_last_ingested_wal_ts() + 1; | ||
gs::WalIngester ingester(graph, cur_ts); | ||
gs::KafkaWalConsumer consumer(ingester, config, topic_name, 1); | ||
while (graph.db().kafka_wal_ingester_state()) { | ||
consumer.poll(); | ||
ingester.ingest(); | ||
std::this_thread::sleep_for(std::chrono::milliseconds(100)); | ||
} | ||
while (!ingester.empty()) { | ||
consumer.poll(); | ||
ingester.ingest(); | ||
} | ||
return true; | ||
} | ||
AppWrapper KafkaWalIngesterAppFactory::CreateApp(const GraphDB& db) { | ||
return AppWrapper(new KafkaWalIngesterApp(), NULL); | ||
} | ||
#endif | ||
} // namespace gs |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,45 @@ | ||
/** Copyright 2020 Alibaba Group Holding Limited. | ||
* | ||
* 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. | ||
*/ | ||
|
||
#ifndef ENGINES_KAFKA_WAL_INGESTER_APP_H_ | ||
#define ENGINES_KAFKA_WAL_INGESTER_APP_H_ | ||
|
||
#include "flex/engines/graph_db/app/app_base.h" | ||
#include "flex/engines/graph_db/database/graph_db_session.h" | ||
|
||
namespace gs { | ||
#ifdef BUILD_KAFKA_WAL_WRITER_PARSER | ||
// Ingest wal from kafka | ||
class KafkaWalIngesterApp : public WriteAppBase { | ||
public: | ||
KafkaWalIngesterApp() {} | ||
|
||
AppType type() const override { return AppType::kBuiltIn; } | ||
|
||
bool Query(GraphDBSession& graph, Decoder& input, Encoder& output) override; | ||
}; | ||
|
||
class KafkaWalIngesterAppFactory : public AppFactoryBase { | ||
public: | ||
KafkaWalIngesterAppFactory() = default; | ||
~KafkaWalIngesterAppFactory() = default; | ||
|
||
AppWrapper CreateApp(const GraphDB& db) override; | ||
}; | ||
#endif | ||
|
||
} // namespace gs | ||
|
||
#endif // ENGINES_KAFKA_WAL_INGESTER_APP_H_ |
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Consider adding a delay (or a readiness check) after starting the Kafka server to ensure it is fully initialized before subsequent commands are executed.
Copilot uses AI. Check for mistakes.