Skip to content

Commit

Permalink
feat: insert stmt support for map data type
Browse files Browse the repository at this point in the history
WIP!
  • Loading branch information
aceforeverd committed Jan 29, 2024
1 parent 7ce7503 commit 0cbbe8b
Show file tree
Hide file tree
Showing 20 changed files with 491 additions and 51 deletions.
7 changes: 2 additions & 5 deletions hybridse/include/codec/fe_schema_codec.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,7 @@
#define HYBRIDSE_INCLUDE_CODEC_FE_SCHEMA_CODEC_H_

#include <cstring>
#include <iostream>
#include <map>
#include <string>
#include <vector>
#include "vm/catalog.h"

namespace hybridse {
Expand Down Expand Up @@ -56,7 +53,7 @@ class SchemaCodec {
if (it->name().size() >= 128) {
return false;
}
uint8_t name_size = (uint8_t)(it->name().size());
uint8_t name_size = static_cast<uint8_t>(it->name().size());
memcpy(cbuffer, static_cast<const void*>(&name_size), 1);
cbuffer += 1;
memcpy(cbuffer, static_cast<const void*>(it->name().c_str()),
Expand All @@ -66,7 +63,7 @@ class SchemaCodec {
return true;
}

static bool Decode(const std::string& buf, vm::Schema* schema) {
static bool Decode(const std::string& buf, codec::Schema* schema) {
if (schema == NULL) return false;
if (buf.size() <= 0) return true;
const char* buffer = buf.c_str();
Expand Down
8 changes: 4 additions & 4 deletions hybridse/include/sdk/base_impl.h
Original file line number Diff line number Diff line change
Expand Up @@ -30,13 +30,13 @@ typedef ::google::protobuf::RepeatedPtrField< ::hybridse::type::TableDef>

class SchemaImpl : public Schema {
public:
explicit SchemaImpl(const vm::Schema& schema);
explicit SchemaImpl(const codec::Schema& schema);
SchemaImpl() {}

~SchemaImpl();

const vm::Schema& GetSchema() const { return schema_; }
inline void SetSchema(const vm::Schema& schema) { schema_ = schema; }
const codec::Schema& GetSchema() const { return schema_; }
inline void SetSchema(const codec::Schema& schema) { schema_ = schema; }
int32_t GetColumnCnt() const;

const std::string& GetColumnName(uint32_t index) const;
Expand All @@ -46,7 +46,7 @@ class SchemaImpl : public Schema {
const bool IsConstant(uint32_t index) const;

private:
vm::Schema schema_;
codec::Schema schema_;
};

class TableImpl : public Table {
Expand Down
2 changes: 1 addition & 1 deletion hybridse/src/codegen/buf_ir_builder.cc
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,7 @@ bool BufNativeIRBuilder::BuildGetStringField(uint32_t col_idx, uint32_t offset,

BufNativeEncoderIRBuilder::BufNativeEncoderIRBuilder(CodeGenContextBase* ctx,
const std::map<uint32_t, NativeValue>* outputs,
const vm::Schema* schema)
const codec::Schema* schema)
: ctx_(ctx),
outputs_(outputs),
schema_(schema),
Expand Down
9 changes: 2 additions & 7 deletions hybridse/src/codegen/buf_ir_builder.h
Original file line number Diff line number Diff line change
Expand Up @@ -25,15 +25,14 @@
#include "codegen/row_ir_builder.h"
#include "codegen/scope_var.h"
#include "codegen/variable_ir_builder.h"
#include "vm/catalog.h"

namespace hybridse {
namespace codegen {

class BufNativeEncoderIRBuilder : public RowEncodeIRBuilder {
public:
BufNativeEncoderIRBuilder(CodeGenContextBase* ctx, const std::map<uint32_t, NativeValue>* outputs,
const vm::Schema* schema);
const codec::Schema* schema);

~BufNativeEncoderIRBuilder() override;

Expand All @@ -55,10 +54,6 @@ class BufNativeEncoderIRBuilder : public RowEncodeIRBuilder {
::llvm::Value* str_addr_space, ::llvm::Value* str_body_offset, uint32_t str_field_idx,
::llvm::Value** output);

// encode SQL map data type into row
base::Status AppendMapVal(const type::ColumnSchema& sc, llvm::Value* i8_ptr, uint32_t field_idx,
const NativeValue& val, llvm::Value* str_addr_space, llvm::Value* str_body_offset,
uint32_t str_field_idx, llvm::Value** next_str_body_offset);
absl::StatusOr<llvm::Function*> GetOrBuildAppendMapFn(const type::ColumnSchema& sc) const;

base::Status AppendHeader(::llvm::Value* i8_ptr, ::llvm::Value* size,
Expand All @@ -74,7 +69,7 @@ class BufNativeEncoderIRBuilder : public RowEncodeIRBuilder {
private:
CodeGenContextBase* ctx_;
const std::map<uint32_t, NativeValue>* outputs_;
const vm::Schema* schema_;
const codec::Schema* schema_;
uint32_t str_field_start_offset_;
// n = offset_vec_[i] is
// schema_[i] is base type (except string): col encode offset in row
Expand Down
149 changes: 149 additions & 0 deletions hybridse/src/codegen/insert_row_builder.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
/**
* Copyright (c) 2024 OpenMLDB authors
*
* 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 "codegen/insert_row_builder.h"

#include <map>
#include <string>
#include <utility>
#include <vector>

#include "absl/status/status.h"
#include "base/fe_status.h"
#include "codegen/buf_ir_builder.h"
#include "codegen/context.h"
#include "codegen/expr_ir_builder.h"
#include "node/node_manager.h"
#include "passes/resolve_fn_and_attrs.h"
#include "udf/default_udf_library.h"
#include "vm/engine.h"
#include "vm/jit_wrapper.h"

namespace hybridse {
namespace codegen {

InsertRowBuilder::InsertRowBuilder(const codec::Schema* schema) : schema_(schema) {}

absl::Status InsertRowBuilder::Init() {
::hybridse::vm::Engine::InitializeGlobalLLVM();

jit_ = std::unique_ptr<vm::HybridSeJitWrapper>(vm::HybridSeJitWrapper::Create());
if (!jit_->Init()) {
jit_ = nullptr;
return absl::InternalError("fail to init jit");
}
if (!vm::HybridSeJitWrapper::InitJitSymbols(jit_.get())) {
jit_ = nullptr;
return absl::InternalError("fail to init jit symbols");
}
return absl::OkStatus();
}

absl::StatusOr<std::shared_ptr<int8_t>> InsertRowBuilder::ComputeRow(const node::ExprListNode* values) {
EnsureInitialized();
return ComputeRow(values->children_);
}

absl::StatusOr<std::shared_ptr<int8_t>> InsertRowBuilder::ComputeRow(absl::Span<node::ExprNode* const> values) {
EnsureInitialized();

std::unique_ptr<llvm::LLVMContext> llvm_ctx = llvm::make_unique<llvm::LLVMContext>();
std::unique_ptr<llvm::Module> llvm_module = llvm::make_unique<llvm::Module>("insert_row_builder", *llvm_ctx);
vm::SchemasContext empty_sc;
node::NodeManager nm;
codec::Schema empty_param_types;
CodeGenContext dump_ctx(llvm_module.get(), &empty_sc, &empty_param_types, &nm);

auto library = udf::DefaultUdfLibrary::get();
node::ExprAnalysisContext expr_ctx(&nm, library, &empty_sc, &empty_param_types);
passes::ResolveFnAndAttrs resolver(&expr_ctx);

std::vector<node::ExprNode*> transformed;
for (auto& expr : values) {
node::ExprNode* out = nullptr;
CHECK_STATUS_TO_ABSL(resolver.VisitExpr(expr, &out));
transformed.push_back(out);
}

std::string fn_name = absl::StrCat("gen_insert_row_", fn_counter_++);
auto fs = BuildFn(&dump_ctx, fn_name, transformed);
CHECK_ABSL_STATUSOR(fs);

llvm::Function* fn = fs.value();

if (!jit_->OptModule(llvm_module.get())) {
return absl::InternalError("fail to optimize module");
}

if (!jit_->AddModule(std::move(llvm_module), std::move(llvm_ctx))) {
return absl::InternalError("add llvm module failed");
}

auto c_fn = jit_->FindFunction(fn->getName());
void (*encode)(int8_t**) = reinterpret_cast<void (*)(int8_t**)>(const_cast<int8_t*>(c_fn));

int8_t* insert_row = nullptr;
encode(&insert_row);

auto managed_row = std::shared_ptr<int8_t>(insert_row, std::free);

return managed_row;
}

absl::StatusOr<llvm::Function*> InsertRowBuilder::BuildFn(CodeGenContext* ctx, llvm::StringRef fn_name,
absl::Span<node::ExprNode* const> values) {
llvm::Function* fn = ctx->GetModule()->getFunction(fn_name);
if (fn == nullptr) {
auto builder = ctx->GetBuilder();
llvm::FunctionType* fnt = llvm::FunctionType::get(builder->getVoidTy(),
{
builder->getInt8PtrTy()->getPointerTo(),
},
false);

fn = llvm::Function::Create(fnt, llvm::GlobalValue::ExternalLinkage, fn_name, ctx->GetModule());
FunctionScopeGuard fg(fn, ctx);

llvm::Value* row_ptr_ptr = fn->arg_begin();

ExprIRBuilder expr_builder(ctx);

std::map<uint32_t, NativeValue> columns;
for (uint32_t i = 0; i < values.size(); ++i) {
auto expr = values[i];

NativeValue out;
auto s = expr_builder.Build(expr, &out);
CHECK_STATUS_TO_ABSL(s);

columns[i] = out;
}

BufNativeEncoderIRBuilder encode_builder(ctx, &columns, schema_);
CHECK_STATUS_TO_ABSL(encode_builder.Init());

encode_builder.BuildEncode(row_ptr_ptr);

builder->CreateRetVoid();
}

return fn;
}

// build the function that transform a single insert row values into encoded row
absl::StatusOr<llvm::Function*> InsertRowBuilder::BuildEncodeFn() { return absl::OkStatus(); }
} // namespace codegen
} // namespace hybridse
67 changes: 67 additions & 0 deletions hybridse/src/codegen/insert_row_builder.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/**
* Copyright (c) 2024 OpenMLDB authors
*
* 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 HYBRIDSE_SRC_CODEGEN_INSERT_ROW_BUILDER_H_
#define HYBRIDSE_SRC_CODEGEN_INSERT_ROW_BUILDER_H_

#include <memory>

#include "absl/status/statusor.h"
#include "codec/fe_row_codec.h"
#include "codegen/context.h"
#include "llvm/IR/Function.h"
#include "node/sql_node.h"
#include "vm/jit_wrapper.h"

namespace hybridse {
namespace codegen {

class InsertRowBuilder {
public:
explicit InsertRowBuilder(const codec::Schema* schema);

absl::Status Init();

// compute the encoded row result for insert statement's single values expression list
//
// currently, expressions in insert values do not expect external source, so unsupported expressions
// will simply fail on resolving.
absl::StatusOr<std::shared_ptr<int8_t>> ComputeRow(absl::Span<node::ExprNode* const> values);

absl::StatusOr<std::shared_ptr<int8_t>> ComputeRow(const node::ExprListNode* values);

private:
void EnsureInitialized() { assert(jit_ && "InsertRowBuilder not initialized"); }

// build the function the will output the row from single insert values
//
// the function is just equivalent to C: `void fn(int8_t**)`.
// BuildFn returns different function with different name on every invocation
absl::StatusOr<llvm::Function*> BuildFn(CodeGenContext* ctx, llvm::StringRef fn_name,
absl::Span<node::ExprNode* const>);

// build the function that transform a single insert row values into encoded row
absl::StatusOr<llvm::Function*> BuildEncodeFn();

// CodeGenContextBase* ctx_;
const codec::Schema* schema_;
std::atomic<uint32_t> fn_counter_ = 0;

std::unique_ptr<vm::HybridSeJitWrapper> jit_;
};
} // namespace codegen
} // namespace hybridse
#endif // HYBRIDSE_SRC_CODEGEN_INSERT_ROW_BUILDER_H_
71 changes: 71 additions & 0 deletions hybridse/src/codegen/insert_row_builder_test.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/**
* Copyright (c) 2024 OpenMLDB authors
*
* 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 "codegen/insert_row_builder.h"

#include <string>

#include "gtest/gtest.h"
#include "node/sql_node.h"
#include "plan/plan_api.h"
#include "vm/sql_ctx.h"

namespace hybridse {
namespace codegen {

class InsertRowBuilderTest : public ::testing::Test {};

TEST_F(InsertRowBuilderTest, encode) {
std::string sql = "insert into t1 values (1, map (1, '12'))";
vm::SqlContext ctx;
ctx.sql = sql;
auto s = plan::PlanAPI::CreatePlanTreeFromScript(&ctx);
ASSERT_TRUE(s.isOK()) << s;

auto* exprlist = dynamic_cast<node::InsertPlanNode*>(ctx.logical_plan.front())->GetInsertNode()->values_[0];

codec::Schema sc;
{
auto col1 = sc.Add();
col1->mutable_schema()->set_base_type(type::kInt32);
col1->set_type(type::kInt32);
}

{
auto col = sc.Add();
auto map_ty = col->mutable_schema()->mutable_map_type();
map_ty->mutable_key_type()->set_base_type(type::kInt32);
map_ty->mutable_value_type()->set_base_type(type::kVarchar);
}

InsertRowBuilder builder(&sc);
{
auto s = builder.Init();
ASSERT_TRUE(s.ok()) << s;
}

auto as = builder.ComputeRow(dynamic_cast<node::ExprListNode*>(exprlist));
ASSERT_TRUE(as.ok()) << as.status();

ASSERT_TRUE(as.value() != nullptr);
}
} // namespace codegen
} // namespace hybridse
//
int main(int argc, char** argv) {
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}
2 changes: 1 addition & 1 deletion hybridse/src/plan/planner.cc
Original file line number Diff line number Diff line change
Expand Up @@ -710,7 +710,7 @@ base::Status SimplePlanner::CreatePlanTree(const NodePointVector &parser_trees,
break;
}
case node::kInsertStmt: {
CHECK_TRUE(is_batch_mode_, common::kPlanError, "Non-support INSERT Op in online serving");
// CHECK_TRUE(is_batch_mode_, common::kPlanError, "Non-support INSERT Op in online serving");
node::PlanNode *insert_plan = nullptr;
CHECK_STATUS(CreateInsertPlan(parser_tree, &insert_plan))
plan_trees.push_back(insert_plan);
Expand Down
Loading

0 comments on commit 0cbbe8b

Please sign in to comment.