1+ /* * Copyright 2020 Alibaba Group Holding Limited.
2+ *
3+ * Licensed under the Apache License, Version 2.0 (the "License");
4+ * you may not use this file except in compliance with the License.
5+ * You may obtain a copy of the License at
6+ *
7+ * http://www.apache.org/licenses/LICENSE-2.0
8+ *
9+ * Unless required by applicable law or agreed to in writing, software
10+ * distributed under the License is distributed on an "AS IS" BASIS,
11+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+ * See the License for the specific language governing permissions and
13+ * limitations under the License.
14+ */
15+ #include " flex/engines/graph_db/database/wal/kafka_wal_session.h"
16+ #include " flex/engines/graph_db/app/kafka_wal_ingester_app.h"
17+ #include " flex/engines/graph_db/database/graph_db.h"
18+ #include " flex/engines/graph_db/database/graph_db_session.h"
19+
20+ namespace gs {
21+ bool KafkaWalSession::open (GraphDBSession& session,
22+ const std::string& wal_uri) {
23+ ingester_ = std::make_unique<KafkaWalIngesterApp>();
24+ std::vector<char > buf;
25+ gs::Encoder encoder (buf);
26+
27+ const std::string prefix = " kafka://" ;
28+ if (wal_uri.find (prefix) != 0 ) {
29+ LOG (ERROR) << " Invalid uri: " << wal_uri;
30+ return false ;
31+ }
32+
33+ std::string hosts_part = wal_uri.substr (prefix.length ());
34+ size_t query_pos = hosts_part.find (' /' );
35+ std::string hosts;
36+ std::string query;
37+ cppkafka::Configuration config;
38+ if (query_pos != std::string::npos) {
39+ hosts = hosts_part.substr (0 , query_pos);
40+ query = hosts_part.substr (query_pos + 1 );
41+ } else {
42+ LOG (ERROR) << " Invalid uri: " << wal_uri;
43+ return false ;
44+ }
45+ std::string kafka_brokers = hosts;
46+ encoder.put_string (" metadata.broker.list" );
47+ encoder.put_string (kafka_brokers);
48+ size_t top_pos = query.find (' ?' );
49+ std::string topic;
50+ if (top_pos != std::string::npos) {
51+ topic = query.substr (0 , top_pos);
52+ query = query.substr (top_pos + 1 );
53+ } else {
54+ topic = query;
55+ }
56+ encoder.put_string (" topic_name" );
57+ encoder.put_string (topic);
58+ std::istringstream query_stream (query);
59+ std::string pair;
60+ while (std::getline (query_stream, pair, ' &' )) {
61+ size_t eq_pos = pair.find (' =' );
62+ if (eq_pos != std::string::npos) {
63+ std::string key = pair.substr (0 , eq_pos);
64+ std::string value = pair.substr (eq_pos + 1 );
65+ encoder.put_string (key);
66+ encoder.put_string (value);
67+ }
68+ }
69+ encoder.put_string (" enable.auto.commit" );
70+ encoder.put_string (" false" );
71+ Decoder decoder (buf.data (), buf.size ());
72+
73+ ingester_thread_ = std::thread ([&]() {
74+ std::vector<char > buf;
75+ gs::Encoder encoder (buf);
76+ ingester_->Query (session, decoder, encoder);
77+ gs::Decoder output (buf.data (), buf.size ());
78+ session.db ().set_last_ingested_wal_ts (output.get_long ());
79+ });
80+ return true ;
81+ }
82+
83+ bool KafkaWalSession::close () {
84+ if (ingester_) {
85+ ingester_->terminal ();
86+ }
87+ if (ingester_thread_.joinable ()) {
88+ ingester_thread_.join ();
89+ }
90+ return true ;
91+ }
92+
93+ } // namespace gs
0 commit comments