[Enhancement] Add scripting ability to BE (#20351)

This PR adds scripting ability to BE, as a way to increase system observability, making testing, debugging and online hot fix issues easier. The script engine uses wren script language and wrenbind17 binding util(it's a small OO language). 
Currently, only a small faction of BE objects & methods are bonded, mainly focusing on StorageEngine(Tablets, Rowsets, TabletUpates). 
Will add more binding for rocksdb metadata, thread/stack dumps, memory/cpu profiling utils in the future.

Sample uses cases:

```
mysql> admin execute on 10004 '
    '> var ts = StorageEngine.get_tablet_infos(-1,-1)
    '> for (t in ts) {
    '>     System.print("%(t.tablet_id) %(t.data_size)")
    '> }
    '> ';
Query OK, 0 rows affected (0.01 sec)
10009 0
10011 0
10013 0
...

mysql> admin execute on 10004 '
    '> var t = StorageEngine.get_tablet(11005)
    '> System.print("info: %(t.updates().toPB().toString())")
    '> ';
Query OK, 0 rows affected (0.00 sec)
info: {"versions":[{"version":{"major":"3","minor":"1"},"creationTime":"1679897711","rowsets":[2],"compaction":{"startVersion":{"major":"3","minor":"0"},"inputs":[0,1],"outputs":[2]}},{"version":{"major":"4","minor":"0"},"creationTime":"1679900665","rowsets":[2,3],"deltas":[3]},{"version":{"major":"5","minor":"0"},"creationTime":"1679900666","rowsets":[2,3,4],"deltas":[4]},{"version":{"major":"6","minor":"0"},"creationTime":"1679900667","rowsets":[2,3,4,5],"deltas":[5]},{"version":{"major":"7","minor":"0"},"creationTime":"1679900667","rowsets":[2,3,4,5,6],"deltas":[6]},{"version":{"major":"8","minor":"0"},"creationTime":"1679900667","rowsets":[2,3,4,5,6,7],"deltas":[7]},{"version":{"major":"9","minor":"0"},"creationTime":"1679900667","rowsets":[2,3,4,5,6,7,8],"deltas":[8]},{"version":{"major":"10","minor":"0"},"creationTime":"1679900668","rowsets":[2,3,4,5,6,7,8,9],"deltas":[9]},{"version":{"major":"11","minor":"0"},"creationTime":"1679900668","rowsets":[2,3,4,5,6,7,8,9,10],"deltas":[10]},{"version":{"major":"12","minor":"0"},"creationTime":"1679900668","rowsets":[2,3,4,5,6,7,8,9,10,11],"deltas":[11]},{"version":{"major":"13","minor":"0"},"creationTime":"1679900668","rowsets":[2,3,4,5,6,7,8,9,10,11,12],"deltas":[12]},{"version":{"major":"13","minor":"1"},"creationTime":"1679900708","rowsets":[13],"compaction":{"startVersion":{"major":"13","minor":"0"},"inputs":[2,3,4,5,6,7,8,9,10,11,12],"outputs":[13]}}],"applyVersion":{"major":"13","minor":"1"},"nextRowsetId":14,"nextLogId":"28"}


mysql> admin execute on 10004 '
    '> for (dir in StorageEngine.get_data_dirs()) {
    '>     System.print("dir: %(dir.path()) rocksdb: %(dir.get_meta().sst_file_size())")
    '>     var result = dir.get_meta().compact().toString()
    '>     System.print("compact %(result) size: %(dir.get_meta().sst_file_size())")
    '> }
    '> ';
Query OK, 0 rows affected (0.00 sec)
dir: /home/decster/projects/starrocks/localrun/be/storage rocksdb: 156484
compact OK size: 12708

```

Signed-off-by: Binglin Chang <decstery@gmail.com>
This commit is contained in:
Binglin Chang 2023-03-31 09:37:28 +08:00 committed by GitHub
parent b02dccb044
commit 8d83da4fdb
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
60 changed files with 17811 additions and 24 deletions

View File

@ -614,6 +614,7 @@ set(STARROCKS_LINK_LIBS
Types
Udf
Util
Script
StarRocksGen
Webserver
TestUtil
@ -861,6 +862,7 @@ add_subdirectory(${SRC_DIR}/udf)
add_subdirectory(${SRC_DIR}/tools)
add_subdirectory(${SRC_DIR}/util)
add_subdirectory(${SRC_DIR}/script)
if (WITH_BENCH STREQUAL "ON")
message(STATUS "Build binary with bench")

View File

@ -19,6 +19,7 @@
#include "common/configbase.h"
#include "gutil/strings/substitute.h"
#include "http/action/update_config_action.h"
#include "script/script.h"
namespace starrocks {
@ -41,10 +42,12 @@ Status handle_set_config(const string& params_str) {
return update_config->update_config(name_itr->value.GetString(), value_itr->value.GetString());
}
Status execute_command(const std::string& command, const std::string& params) {
LOG(INFO) << "execute command: " << command << " params: " << params;
Status execute_command(const std::string& command, const std::string& params, std::string* result) {
LOG(INFO) << "execute command: " << command << " params: " << params.substr(0, 2000);
if (command == "set_config") {
return handle_set_config(params);
} else if (command == "execute_script") {
return execute_script(params, *result);
}
return Status::NotSupported(strings::Substitute("command $0 not supported", command));
}

View File

@ -20,6 +20,6 @@
namespace starrocks {
Status execute_command(const std::string& command, const std::string& params);
Status execute_command(const std::string& command, const std::string& params, std::string* result);
} // namespace starrocks

View File

@ -0,0 +1,26 @@
# Copyright 2021-present StarRocks, Inc. All rights reserved.
#
# 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
#
# https://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.
set(LIBRARY_OUTPUT_PATH "${BUILD_DIR}/src/script")
set(EXECUTABLE_OUTPUT_PATH "${BUILD_DIR}/src/script")
set(EXEC_FILES
script.cpp
../thirdparty/wren/wren.c
)
add_library(Script STATIC ${EXEC_FILES})
target_include_directories(Script PRIVATE ${SRC_DIR}/script ${SRC_DIR}/thirdparty/wren ${SRC_DIR}/thirdparty/wrenbind17)

307
be/src/script/script.cpp Normal file
View File

@ -0,0 +1,307 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// 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
//
// https://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 "script/script.h"
#include <google/protobuf/util/json_util.h>
#include "common/logging.h"
#include "exec/schema_scanner/schema_be_tablets_scanner.h"
#include "gen_cpp/olap_file.pb.h"
#include "gutil/strings/substitute.h"
#include "storage/storage_engine.h"
#include "storage/tablet.h"
#include "storage/tablet_manager.h"
#include "storage/tablet_updates.h"
#include "wrenbind17/wrenbind17.hpp"
using namespace wrenbind17;
using std::string;
namespace starrocks {
#define REG_VAR(TYPE, NAME) cls.var<&TYPE::NAME>(#NAME)
#define REG_METHOD(TYPE, NAME) cls.func<&TYPE::NAME>(#NAME)
#define REG_STATIC_METHOD(TYPE, NAME) cls.funcStatic<&TYPE::NAME>(#NAME)
template <class T>
std::string proto_to_json(T& proto) {
std::string json;
google::protobuf::util::MessageToJsonString(proto, &json);
return json;
}
static std::shared_ptr<TabletUpdatesPB> tablet_updates_to_pb(TabletUpdates& self) {
std::shared_ptr<TabletUpdatesPB> pb = std::make_shared<TabletUpdatesPB>();
self.to_updates_pb(pb.get());
return pb;
}
static EditVersionMetaPB* tablet_updates_pb_version(TabletUpdatesPB& self, int idx) {
if (idx < 0 || idx >= self.versions_size()) return nullptr;
return self.mutable_versions(idx);
}
static uint64_t kv_store_get_live_sst_files_size(KVStore& store) {
uint64_t ret = 0;
store.get_live_sst_files_size(&ret);
return ret;
}
static int tablet_keys_type_int(Tablet& tablet) {
return static_cast<int>(tablet.keys_type());
}
static int tablet_tablet_state(Tablet& tablet) {
return static_cast<int>(tablet.tablet_state());
}
static const TabletSchema& tablet_tablet_schema(Tablet& tablet) {
return tablet.tablet_schema();
}
static uint64_t tablet_tablet_id(Tablet& tablet) {
return tablet.tablet_id();
}
static std::string tablet_path(Tablet& tablet) {
return tablet.schema_hash_path();
}
static DataDir* tablet_data_dir(Tablet& tablet) {
return tablet.data_dir();
}
static uint64_t get_major(EditVersion& self) {
return self.major();
}
static uint64_t get_minor(EditVersion& self) {
return self.minor();
}
static void bind_common(ForeignModule& m) {
{
auto& cls = m.klass<Status>("Status");
cls.func<&Status::to_string>("toString");
REG_METHOD(Status, ok);
}
}
class StorageEngineRef {
public:
static string drop_tablet(int64_t tablet_id) {
auto manager = StorageEngine::instance()->tablet_manager();
string err;
auto ptr = manager->get_tablet(tablet_id, true, &err);
if (ptr == nullptr) {
return strings::Substitute("get tablet $0 failed: $1", tablet_id, err);
}
auto st = manager->drop_tablet(tablet_id, TabletDropFlag::kKeepMetaAndFiles);
return st.to_string();
}
static std::shared_ptr<Tablet> get_tablet(int64_t tablet_id) {
string err;
auto ptr = StorageEngine::instance()->tablet_manager()->get_tablet(tablet_id, true, &err);
if (ptr == nullptr) {
LOG(WARNING) << "get_tablet " << tablet_id << " failed: " << err;
return nullptr;
}
return ptr;
}
static std::shared_ptr<TabletBasicInfo> get_tablet_info(int64_t tablet_id) {
std::vector<TabletBasicInfo> tablet_infos;
auto manager = StorageEngine::instance()->tablet_manager();
manager->get_tablets_basic_infos(-1, -1, tablet_id, tablet_infos);
if (tablet_infos.empty()) {
return nullptr;
} else {
return std::make_shared<TabletBasicInfo>(tablet_infos[0]);
}
}
static std::vector<TabletBasicInfo> get_tablet_infos(int64_t table_id, int64_t partition_id) {
std::vector<TabletBasicInfo> tablet_infos;
auto manager = StorageEngine::instance()->tablet_manager();
manager->get_tablets_basic_infos(table_id, partition_id, -1, tablet_infos);
return tablet_infos;
}
static std::vector<DataDir*> get_data_dirs() { return StorageEngine::instance()->get_stores(); }
static void bind(ForeignModule& m) {
{
auto& cls = m.klass<TabletBasicInfo>("TabletBasicInfo");
REG_VAR(TabletBasicInfo, table_id);
REG_VAR(TabletBasicInfo, partition_id);
REG_VAR(TabletBasicInfo, tablet_id);
REG_VAR(TabletBasicInfo, num_version);
REG_VAR(TabletBasicInfo, max_version);
REG_VAR(TabletBasicInfo, min_version);
REG_VAR(TabletBasicInfo, num_rowset);
REG_VAR(TabletBasicInfo, num_row);
REG_VAR(TabletBasicInfo, data_size);
REG_VAR(TabletBasicInfo, index_mem);
REG_VAR(TabletBasicInfo, create_time);
REG_VAR(TabletBasicInfo, state);
REG_VAR(TabletBasicInfo, type);
}
{
auto& cls = m.klass<TabletSchema>("TabletSchema");
REG_METHOD(TabletSchema, num_columns);
REG_METHOD(TabletSchema, num_key_columns);
REG_METHOD(TabletSchema, keys_type);
REG_METHOD(TabletSchema, mem_usage);
cls.func<&TabletSchema::debug_string>("toString");
}
{
auto& cls = m.klass<Tablet>("Tablet");
cls.funcExt<tablet_tablet_id>("tablet_id");
cls.funcExt<tablet_tablet_schema>("schema");
cls.funcExt<tablet_path>("path");
cls.funcExt<tablet_data_dir>("data_dir");
cls.funcExt<tablet_keys_type_int>("keys_type_as_int");
cls.funcExt<tablet_tablet_state>("tablet_state_as_int");
REG_METHOD(Tablet, tablet_footprint);
REG_METHOD(Tablet, num_rows);
REG_METHOD(Tablet, version_count);
REG_METHOD(Tablet, max_version);
REG_METHOD(Tablet, max_continuous_version);
REG_METHOD(Tablet, compaction_score);
REG_METHOD(Tablet, schema_debug_string);
REG_METHOD(Tablet, debug_string);
REG_METHOD(Tablet, support_binlog);
REG_METHOD(Tablet, updates);
}
{
auto& cls = m.klass<EditVersionPB>("EditVersionPB");
cls.funcExt<&proto_to_json<EditVersionPB>>("toString");
}
{
auto& cls = m.klass<EditVersionMetaPB>("EditVersionMetaPB");
REG_METHOD(EditVersionMetaPB, version);
REG_METHOD(EditVersionMetaPB, creation_time);
cls.funcExt<&proto_to_json<EditVersionMetaPB>>("toString");
}
{
auto& cls = m.klass<TabletUpdatesPB>("TabletUpdatesPB");
REG_METHOD(TabletUpdatesPB, versions_size);
cls.funcExt<&tablet_updates_pb_version>("versions");
REG_METHOD(TabletUpdatesPB, apply_version);
REG_METHOD(TabletUpdatesPB, next_rowset_id);
REG_METHOD(TabletUpdatesPB, next_log_id);
cls.funcExt<&proto_to_json<TabletUpdatesPB>>("toString");
}
{
auto& cls = m.klass<EditVersion>("EditVersion");
cls.funcExt<&get_major>("major");
cls.funcExt<&get_minor>("minor");
cls.func<&EditVersion::to_string>("toString");
}
{
auto& cls = m.klass<CompactionInfo>("CompactionInfo");
REG_VAR(CompactionInfo, start_version);
REG_VAR(CompactionInfo, inputs);
REG_VAR(CompactionInfo, output);
}
{
auto& cls = m.klass<EditVersionInfo>("EditVersionInfo");
REG_VAR(EditVersionInfo, version);
REG_VAR(EditVersionInfo, creation_time);
REG_VAR(EditVersionInfo, rowsets);
REG_VAR(EditVersionInfo, deltas);
REG_METHOD(EditVersionInfo, get_compaction);
}
{
auto& cls = m.klass<Rowset>("Rowset");
REG_METHOD(Rowset, rowset_id_str);
REG_METHOD(Rowset, schema);
REG_METHOD(Rowset, start_version);
REG_METHOD(Rowset, end_version);
REG_METHOD(Rowset, creation_time);
REG_METHOD(Rowset, data_disk_size);
REG_METHOD(Rowset, empty);
REG_METHOD(Rowset, num_rows);
REG_METHOD(Rowset, total_row_size);
REG_METHOD(Rowset, txn_id);
REG_METHOD(Rowset, partition_id);
REG_METHOD(Rowset, num_segments);
REG_METHOD(Rowset, num_delete_files);
REG_METHOD(Rowset, rowset_path);
}
{
auto& cls = m.klass<TabletUpdates>("TabletUpdates");
REG_METHOD(TabletUpdates, get_error_msg);
REG_METHOD(TabletUpdates, num_rows);
REG_METHOD(TabletUpdates, data_size);
REG_METHOD(TabletUpdates, num_rowsets);
REG_METHOD(TabletUpdates, max_version);
REG_METHOD(TabletUpdates, version_count);
REG_METHOD(TabletUpdates, num_pending);
REG_METHOD(TabletUpdates, get_compaction_score);
REG_METHOD(TabletUpdates, version_history_count);
REG_METHOD(TabletUpdates, get_average_row_size);
REG_METHOD(TabletUpdates, debug_string);
REG_METHOD(TabletUpdates, get_compaction_score);
REG_METHOD(TabletUpdates, get_version_list);
REG_METHOD(TabletUpdates, get_edit_version);
REG_METHOD(TabletUpdates, get_rowset_map);
cls.funcExt<&tablet_updates_to_pb>("toPB");
}
{
auto& cls = m.klass<DataDir>("DataDir");
REG_METHOD(DataDir, path);
REG_METHOD(DataDir, path_hash);
REG_METHOD(DataDir, is_used);
REG_METHOD(DataDir, get_meta);
REG_METHOD(DataDir, is_used);
REG_METHOD(DataDir, available_bytes);
REG_METHOD(DataDir, disk_capacity_bytes);
}
{
auto& cls = m.klass<KVStore>("KVStore");
REG_METHOD(KVStore, compact);
REG_METHOD(KVStore, flushMemTable);
REG_METHOD(KVStore, get_stats);
cls.funcExt<&kv_store_get_live_sst_files_size>("sst_file_size");
}
{
auto& cls = m.klass<StorageEngineRef>("StorageEngine");
REG_STATIC_METHOD(StorageEngineRef, get_tablet_info);
REG_STATIC_METHOD(StorageEngineRef, get_tablet_infos);
REG_STATIC_METHOD(StorageEngineRef, get_tablet);
REG_STATIC_METHOD(StorageEngineRef, drop_tablet);
REG_STATIC_METHOD(StorageEngineRef, get_data_dirs);
}
}
};
Status execute_script(const std::string& script, std::string& output) {
wrenbind17::VM vm;
vm.setPrintFunc([&](const char* text) { output.append(text); });
auto& m = vm.module("starrocks");
bind_common(m);
StorageEngineRef::bind(m);
vm.runFromSource("main", R"(import "starrocks" for StorageEngine)");
try {
vm.runFromSource("main", script);
} catch (const std::exception& e) {
output.append(e.what());
}
return Status::OK();
}
} // namespace starrocks

25
be/src/script/script.h Normal file
View File

@ -0,0 +1,25 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// 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
//
// https://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 <common/status.h>
#include <string>
namespace starrocks {
Status execute_script(const std::string& script, std::string& output);
} // namespace starrocks

View File

@ -963,7 +963,7 @@ void PInternalServiceImplBase<T>::execute_command(google::protobuf::RpcControlle
const ExecuteCommandRequestPB* request,
ExecuteCommandResultPB* response, google::protobuf::Closure* done) {
ClosureGuard closure_guard(done);
Status st = starrocks::execute_command(request->command(), request->params());
Status st = starrocks::execute_command(request->command(), request->params(), response->mutable_result());
if (!st.ok()) {
LOG(WARNING) << "execute_command failed, errmsg=" << st.to_string();
}

View File

@ -87,6 +87,9 @@ public:
return info;
}
int64_t available_bytes() const { return _available_bytes; }
int64_t disk_capacity_bytes() const { return _disk_capacity_bytes; }
// save a cluster_id file under data path to prevent
// invalid be config for example two be use the same
// data path

View File

@ -286,10 +286,14 @@ Status KVStore::compact() {
return to_status(st);
}
Status KVStore::flush() {
Status KVStore::flushWAL() {
return to_status(_db->FlushWAL(true));
}
Status KVStore::flushMemTable() {
return to_status(_db->Flush(rocksdb::FlushOptions()));
}
std::string KVStore::get_stats() {
rocksdb::ColumnFamilyHandle* handle = _handles[META_COLUMN_FAMILY_INDEX];
std::string stats;

View File

@ -77,9 +77,13 @@ public:
const std::string& upper_bound,
std::function<bool(std::string_view, std::string_view)> const& func);
const std::string& root_path() const { return _root_path; }
Status compact();
Status flush();
Status flushWAL();
Status flushMemTable();
std::string get_stats();

View File

@ -209,6 +209,7 @@ public:
size_t total_row_size() const { return rowset_meta()->total_row_size(); }
Version version() const { return rowset_meta()->version(); }
RowsetId rowset_id() const { return rowset_meta()->rowset_id(); }
std::string rowset_id_str() const { return rowset_meta()->rowset_id().to_string(); }
int64_t creation_time() const { return rowset_meta()->creation_time(); }
PUniqueId load_id() const { return rowset_meta()->load_id(); }
int64_t txn_id() const { return rowset_meta()->txn_id(); }

View File

@ -67,7 +67,7 @@ Status RowsetMetaManager::save(KVStore* meta, const TabletUid& tablet_uid, const
}
Status RowsetMetaManager::flush(KVStore* meta) {
return meta->flush();
return meta->flushWAL();
}
Status RowsetMetaManager::remove(KVStore* meta, const TabletUid& tablet_uid, const RowsetId& rowset_id) {

View File

@ -426,6 +426,17 @@ Status StorageEngine::set_cluster_id(int32_t cluster_id) {
return Status::OK();
}
std::vector<string> StorageEngine::get_store_paths() {
std::vector<string> paths;
{
std::lock_guard<std::mutex> l(_store_lock);
for (auto& it : _store_map) {
paths.push_back(it.first);
}
}
return paths;
}
std::vector<DataDir*> StorageEngine::get_stores_for_create_tablet(TStorageMedium::type storage_medium) {
std::vector<DataDir*> stores;
{

View File

@ -116,6 +116,7 @@ public:
Status get_all_data_dir_info(std::vector<DataDirInfo>* data_dir_infos, bool need_update);
std::vector<string> get_store_paths();
// get root path for creating tablet. The returned vector of root path should be random,
// for avoiding that all the tablet would be deployed one disk.
std::vector<DataDir*> get_stores_for_create_tablet(TStorageMedium::type storage_medium);

View File

@ -1396,4 +1396,16 @@ void Tablet::get_basic_info(TabletBasicInfo& info) {
}
}
std::string Tablet::schema_debug_string() const {
return _tablet_meta->tablet_schema().debug_string();
}
std::string Tablet::debug_string() const {
if (_updates) {
return _updates->debug_string();
}
// TODO: add more debug info
return string();
}
} // namespace starrocks

View File

@ -117,6 +117,8 @@ public:
size_t num_rows_per_row_block() const;
size_t next_unique_id() const;
size_t field_index(const string& field_name) const;
std::string schema_debug_string() const;
std::string debug_string() const;
// operation in rowsets
Status add_rowset(const RowsetSharedPtr& rowset, bool need_persist = true);

View File

@ -3721,4 +3721,28 @@ void TabletUpdates::to_rowset_meta_pb(const std::vector<RowsetMetaSharedPtr>& ro
}
}
std::vector<std::string> TabletUpdates::get_version_list() const {
std::lock_guard wl(_lock);
std::vector<std::string> version_list;
for (auto& edit_version_info : _edit_version_infos) {
version_list.emplace_back(edit_version_info->version.to_string());
}
return version_list;
}
std::shared_ptr<EditVersionInfo> TabletUpdates::get_edit_version(const string& version) const {
std::lock_guard wl(_lock);
for (auto& edit_version_info : _edit_version_infos) {
if (edit_version_info->version.to_string() == version) {
return std::make_shared<EditVersionInfo>(*edit_version_info);
}
}
return nullptr;
}
std::shared_ptr<std::unordered_map<uint32_t, RowsetSharedPtr>> TabletUpdates::get_rowset_map() const {
std::lock_guard lg(_rowsets_lock);
return std::make_shared<std::unordered_map<uint32_t, RowsetSharedPtr>>(_rowsets);
}
} // namespace starrocks

View File

@ -55,6 +55,30 @@ struct CompactionInfo {
uint32_t output = UINT32_MAX;
};
struct EditVersionInfo {
EditVersion version;
int64_t creation_time;
std::vector<uint32_t> rowsets;
// used for rowset commit
std::vector<uint32_t> deltas;
// used for compaction commit
std::unique_ptr<CompactionInfo> compaction;
EditVersionInfo() = default;
// add a copy constructor to better expose to scripting engine
EditVersionInfo(const EditVersionInfo& rhs) {
version = rhs.version;
creation_time = rhs.creation_time;
rowsets = rhs.rowsets;
deltas = rhs.deltas;
if (rhs.compaction) {
compaction = std::make_unique<CompactionInfo>();
*compaction = *rhs.compaction;
}
}
// add method to better expose to scripting engine
CompactionInfo* get_compaction() { return compaction.get(); }
};
// maintain all states for updatable tablets
class TabletUpdates {
public:
@ -260,6 +284,13 @@ public:
void get_basic_info_extra(TabletBasicInfo& info);
// methods used by scripting engine
std::vector<std::string> get_version_list() const;
std::shared_ptr<EditVersionInfo> get_edit_version(const string& version) const;
std::shared_ptr<std::unordered_map<uint32_t, RowsetSharedPtr>> get_rowset_map() const;
private:
friend class Tablet;
friend class PrimaryIndex;
@ -269,16 +300,6 @@ private:
template <typename K, typename V>
using OrderedMap = std::map<K, V>;
struct EditVersionInfo {
EditVersion version;
int64_t creation_time;
std::vector<uint32_t> rowsets;
// used for rowset commit
std::vector<uint32_t> deltas;
// used for compaction commit
std::unique_ptr<CompactionInfo> compaction;
};
struct RowsetStats {
size_t num_segments = 0;
size_t num_rows = 0;

View File

@ -368,7 +368,7 @@ Status TxnManager::persist_tablet_related_txns(const std::vector<TabletSharedPtr
int i = 0;
for (auto& tablet : to_flush_tablet) {
auto dir = tablet->data_dir();
token->submit_func([&pair_vec, dir, i]() { pair_vec[i].first = dir->get_meta()->flush(); });
token->submit_func([&pair_vec, dir, i]() { pair_vec[i].first = dir->get_meta()->flushWAL(); });
pair_vec[i].second = tablet->tablet_id();
i++;
}
@ -395,7 +395,7 @@ void TxnManager::flush_dirs(std::unordered_set<DataDir*>& affected_dirs) {
std::vector<std::pair<Status, std::string>> pair_vec(affected_dirs.size());
auto token = _flush_thread_pool->new_token(ThreadPool::ExecutionMode::CONCURRENT);
for (auto dir : affected_dirs) {
token->submit_func([&pair_vec, dir, i]() { pair_vec[i].first = dir->get_meta()->flush(); });
token->submit_func([&pair_vec, dir, i]() { pair_vec[i].first = dir->get_meta()->flushWAL(); });
pair_vec[i].second = dir->path();
i++;
}

12266
be/src/thirdparty/wren/wren.c vendored Normal file

File diff suppressed because it is too large Load Diff

534
be/src/thirdparty/wren/wren.h vendored Normal file
View File

@ -0,0 +1,534 @@
#ifndef wren_h
#define wren_h
#include <stdarg.h>
#include <stdbool.h>
#include <stdlib.h>
// The Wren semantic version number components.
#define WREN_VERSION_MAJOR 0
#define WREN_VERSION_MINOR 4
#define WREN_VERSION_PATCH 0
// A human-friendly string representation of the version.
#define WREN_VERSION_STRING "0.4.0"
// A monotonically increasing numeric representation of the version number. Use
// this if you want to do range checks over versions.
#define WREN_VERSION_NUMBER (WREN_VERSION_MAJOR * 1000000 + WREN_VERSION_MINOR * 1000 + WREN_VERSION_PATCH)
#ifndef WREN_API
#if defined(_MSC_VER) && defined(WREN_API_DLLEXPORT)
#define WREN_API __declspec(dllexport)
#else
#define WREN_API
#endif
#endif //WREN_API
// A single virtual machine for executing Wren code.
//
// Wren has no global state, so all state stored by a running interpreter lives
// here.
typedef struct WrenVM WrenVM;
// A handle to a Wren object.
//
// This lets code outside of the VM hold a persistent reference to an object.
// After a handle is acquired, and until it is released, this ensures the
// garbage collector will not reclaim the object it references.
typedef struct WrenHandle WrenHandle;
// A generic allocation function that handles all explicit memory management
// used by Wren. It's used like so:
//
// - To allocate new memory, [memory] is NULL and [newSize] is the desired
// size. It should return the allocated memory or NULL on failure.
//
// - To attempt to grow an existing allocation, [memory] is the memory, and
// [newSize] is the desired size. It should return [memory] if it was able to
// grow it in place, or a new pointer if it had to move it.
//
// - To shrink memory, [memory] and [newSize] are the same as above but it will
// always return [memory].
//
// - To free memory, [memory] will be the memory to free and [newSize] will be
// zero. It should return NULL.
typedef void* (*WrenReallocateFn)(void* memory, size_t newSize, void* userData);
// A function callable from Wren code, but implemented in C.
typedef void (*WrenForeignMethodFn)(WrenVM* vm);
// A finalizer function for freeing resources owned by an instance of a foreign
// class. Unlike most foreign methods, finalizers do not have access to the VM
// and should not interact with it since it's in the middle of a garbage
// collection.
typedef void (*WrenFinalizerFn)(void* data);
// Gives the host a chance to canonicalize the imported module name,
// potentially taking into account the (previously resolved) name of the module
// that contains the import. Typically, this is used to implement relative
// imports.
typedef const char* (*WrenResolveModuleFn)(WrenVM* vm, const char* importer, const char* name);
// Forward declare
struct WrenLoadModuleResult;
// Called after loadModuleFn is called for module [name]. The original returned result
// is handed back to you in this callback, so that you can free memory if appropriate.
typedef void (*WrenLoadModuleCompleteFn)(WrenVM* vm, const char* name, struct WrenLoadModuleResult result);
// The result of a loadModuleFn call.
// [source] is the source code for the module, or NULL if the module is not found.
// [onComplete] an optional callback that will be called once Wren is done with the result.
typedef struct WrenLoadModuleResult {
const char* source;
WrenLoadModuleCompleteFn onComplete;
void* userData;
} WrenLoadModuleResult;
// Loads and returns the source code for the module [name].
typedef WrenLoadModuleResult (*WrenLoadModuleFn)(WrenVM* vm, const char* name);
// Returns a pointer to a foreign method on [className] in [module] with
// [signature].
typedef WrenForeignMethodFn (*WrenBindForeignMethodFn)(WrenVM* vm, const char* module, const char* className,
bool isStatic, const char* signature);
// Displays a string of text to the user.
typedef void (*WrenWriteFn)(WrenVM* vm, const char* text);
typedef enum {
// A syntax or resolution error detected at compile time.
WREN_ERROR_COMPILE,
// The error message for a runtime error.
WREN_ERROR_RUNTIME,
// One entry of a runtime error's stack trace.
WREN_ERROR_STACK_TRACE
} WrenErrorType;
// Reports an error to the user.
//
// An error detected during compile time is reported by calling this once with
// [type] `WREN_ERROR_COMPILE`, the resolved name of the [module] and [line]
// where the error occurs, and the compiler's error [message].
//
// A runtime error is reported by calling this once with [type]
// `WREN_ERROR_RUNTIME`, no [module] or [line], and the runtime error's
// [message]. After that, a series of [type] `WREN_ERROR_STACK_TRACE` calls are
// made for each line in the stack trace. Each of those has the resolved
// [module] and [line] where the method or function is defined and [message] is
// the name of the method or function.
typedef void (*WrenErrorFn)(WrenVM* vm, WrenErrorType type, const char* module, int line, const char* message);
typedef struct {
// The callback invoked when the foreign object is created.
//
// This must be provided. Inside the body of this, it must call
// [wrenSetSlotNewForeign()] exactly once.
WrenForeignMethodFn allocate;
// The callback invoked when the garbage collector is about to collect a
// foreign object's memory.
//
// This may be `NULL` if the foreign class does not need to finalize.
WrenFinalizerFn finalize;
} WrenForeignClassMethods;
// Returns a pair of pointers to the foreign methods used to allocate and
// finalize the data for instances of [className] in resolved [module].
typedef WrenForeignClassMethods (*WrenBindForeignClassFn)(WrenVM* vm, const char* module, const char* className);
typedef struct {
// The callback Wren will use to allocate, reallocate, and deallocate memory.
//
// If `NULL`, defaults to a built-in function that uses `realloc` and `free`.
WrenReallocateFn reallocateFn;
// The callback Wren uses to resolve a module name.
//
// Some host applications may wish to support "relative" imports, where the
// meaning of an import string depends on the module that contains it. To
// support that without baking any policy into Wren itself, the VM gives the
// host a chance to resolve an import string.
//
// Before an import is loaded, it calls this, passing in the name of the
// module that contains the import and the import string. The host app can
// look at both of those and produce a new "canonical" string that uniquely
// identifies the module. This string is then used as the name of the module
// going forward. It is what is passed to [loadModuleFn], how duplicate
// imports of the same module are detected, and how the module is reported in
// stack traces.
//
// If you leave this function NULL, then the original import string is
// treated as the resolved string.
//
// If an import cannot be resolved by the embedder, it should return NULL and
// Wren will report that as a runtime error.
//
// Wren will take ownership of the string you return and free it for you, so
// it should be allocated using the same allocation function you provide
// above.
WrenResolveModuleFn resolveModuleFn;
// The callback Wren uses to load a module.
//
// Since Wren does not talk directly to the file system, it relies on the
// embedder to physically locate and read the source code for a module. The
// first time an import appears, Wren will call this and pass in the name of
// the module being imported. The method will return a result, which contains
// the source code for that module. Memory for the source is owned by the
// host application, and can be freed using the onComplete callback.
//
// This will only be called once for any given module name. Wren caches the
// result internally so subsequent imports of the same module will use the
// previous source and not call this.
//
// If a module with the given name could not be found by the embedder, it
// should return NULL and Wren will report that as a runtime error.
WrenLoadModuleFn loadModuleFn;
// The callback Wren uses to find a foreign method and bind it to a class.
//
// When a foreign method is declared in a class, this will be called with the
// foreign method's module, class, and signature when the class body is
// executed. It should return a pointer to the foreign function that will be
// bound to that method.
//
// If the foreign function could not be found, this should return NULL and
// Wren will report it as runtime error.
WrenBindForeignMethodFn bindForeignMethodFn;
// The callback Wren uses to find a foreign class and get its foreign methods.
//
// When a foreign class is declared, this will be called with the class's
// module and name when the class body is executed. It should return the
// foreign functions uses to allocate and (optionally) finalize the bytes
// stored in the foreign object when an instance is created.
WrenBindForeignClassFn bindForeignClassFn;
// The callback Wren uses to display text when `System.print()` or the other
// related functions are called.
//
// If this is `NULL`, Wren discards any printed text.
WrenWriteFn writeFn;
// The callback Wren uses to report errors.
//
// When an error occurs, this will be called with the module name, line
// number, and an error message. If this is `NULL`, Wren doesn't report any
// errors.
WrenErrorFn errorFn;
// The number of bytes Wren will allocate before triggering the first garbage
// collection.
//
// If zero, defaults to 10MB.
size_t initialHeapSize;
// After a collection occurs, the threshold for the next collection is
// determined based on the number of bytes remaining in use. This allows Wren
// to shrink its memory usage automatically after reclaiming a large amount
// of memory.
//
// This can be used to ensure that the heap does not get too small, which can
// in turn lead to a large number of collections afterwards as the heap grows
// back to a usable size.
//
// If zero, defaults to 1MB.
size_t minHeapSize;
// Wren will resize the heap automatically as the number of bytes
// remaining in use after a collection changes. This number determines the
// amount of additional memory Wren will use after a collection, as a
// percentage of the current heap size.
//
// For example, say that this is 50. After a garbage collection, when there
// are 400 bytes of memory still in use, the next collection will be triggered
// after a total of 600 bytes are allocated (including the 400 already in
// use.)
//
// Setting this to a smaller number wastes less memory, but triggers more
// frequent garbage collections.
//
// If zero, defaults to 50.
int heapGrowthPercent;
// User-defined data associated with the VM.
void* userData;
} WrenConfiguration;
typedef enum { WREN_RESULT_SUCCESS, WREN_RESULT_COMPILE_ERROR, WREN_RESULT_RUNTIME_ERROR } WrenInterpretResult;
// The type of an object stored in a slot.
//
// This is not necessarily the object's *class*, but instead its low level
// representation type.
typedef enum {
WREN_TYPE_BOOL,
WREN_TYPE_NUM,
WREN_TYPE_FOREIGN,
WREN_TYPE_LIST,
WREN_TYPE_MAP,
WREN_TYPE_NULL,
WREN_TYPE_STRING,
// The object is of a type that isn't accessible by the C API.
WREN_TYPE_UNKNOWN
} WrenType;
// Get the current wren version number.
//
// Can be used to range checks over versions.
WREN_API int wrenGetVersionNumber();
// Initializes [configuration] with all of its default values.
//
// Call this before setting the particular fields you care about.
WREN_API void wrenInitConfiguration(WrenConfiguration* configuration);
// Creates a new Wren virtual machine using the given [configuration]. Wren
// will copy the configuration data, so the argument passed to this can be
// freed after calling this. If [configuration] is `NULL`, uses a default
// configuration.
WREN_API WrenVM* wrenNewVM(WrenConfiguration* configuration);
// Disposes of all resources is use by [vm], which was previously created by a
// call to [wrenNewVM].
WREN_API void wrenFreeVM(WrenVM* vm);
// Immediately run the garbage collector to free unused memory.
WREN_API void wrenCollectGarbage(WrenVM* vm);
// Runs [source], a string of Wren source code in a new fiber in [vm] in the
// context of resolved [module].
WREN_API WrenInterpretResult wrenInterpret(WrenVM* vm, const char* module, const char* source);
// Creates a handle that can be used to invoke a method with [signature] on
// using a receiver and arguments that are set up on the stack.
//
// This handle can be used repeatedly to directly invoke that method from C
// code using [wrenCall].
//
// When you are done with this handle, it must be released using
// [wrenReleaseHandle].
WREN_API WrenHandle* wrenMakeCallHandle(WrenVM* vm, const char* signature);
// Calls [method], using the receiver and arguments previously set up on the
// stack.
//
// [method] must have been created by a call to [wrenMakeCallHandle]. The
// arguments to the method must be already on the stack. The receiver should be
// in slot 0 with the remaining arguments following it, in order. It is an
// error if the number of arguments provided does not match the method's
// signature.
//
// After this returns, you can access the return value from slot 0 on the stack.
WREN_API WrenInterpretResult wrenCall(WrenVM* vm, WrenHandle* method);
// Releases the reference stored in [handle]. After calling this, [handle] can
// no longer be used.
WREN_API void wrenReleaseHandle(WrenVM* vm, WrenHandle* handle);
// The following functions are intended to be called from foreign methods or
// finalizers. The interface Wren provides to a foreign method is like a
// register machine: you are given a numbered array of slots that values can be
// read from and written to. Values always live in a slot (unless explicitly
// captured using wrenGetSlotHandle(), which ensures the garbage collector can
// find them.
//
// When your foreign function is called, you are given one slot for the receiver
// and each argument to the method. The receiver is in slot 0 and the arguments
// are in increasingly numbered slots after that. You are free to read and
// write to those slots as you want. If you want more slots to use as scratch
// space, you can call wrenEnsureSlots() to add more.
//
// When your function returns, every slot except slot zero is discarded and the
// value in slot zero is used as the return value of the method. If you don't
// store a return value in that slot yourself, it will retain its previous
// value, the receiver.
//
// While Wren is dynamically typed, C is not. This means the C interface has to
// support the various types of primitive values a Wren variable can hold: bool,
// double, string, etc. If we supported this for every operation in the C API,
// there would be a combinatorial explosion of functions, like "get a
// double-valued element from a list", "insert a string key and double value
// into a map", etc.
//
// To avoid that, the only way to convert to and from a raw C value is by going
// into and out of a slot. All other functions work with values already in a
// slot. So, to add an element to a list, you put the list in one slot, and the
// element in another. Then there is a single API function wrenInsertInList()
// that takes the element out of that slot and puts it into the list.
//
// The goal of this API is to be easy to use while not compromising performance.
// The latter means it does not do type or bounds checking at runtime except
// using assertions which are generally removed from release builds. C is an
// unsafe language, so it's up to you to be careful to use it correctly. In
// return, you get a very fast FFI.
// Returns the number of slots available to the current foreign method.
WREN_API int wrenGetSlotCount(WrenVM* vm);
// Ensures that the foreign method stack has at least [numSlots] available for
// use, growing the stack if needed.
//
// Does not shrink the stack if it has more than enough slots.
//
// It is an error to call this from a finalizer.
WREN_API void wrenEnsureSlots(WrenVM* vm, int numSlots);
// Gets the type of the object in [slot].
WREN_API WrenType wrenGetSlotType(WrenVM* vm, int slot);
// Reads a boolean value from [slot].
//
// It is an error to call this if the slot does not contain a boolean value.
WREN_API bool wrenGetSlotBool(WrenVM* vm, int slot);
// Reads a byte array from [slot].
//
// The memory for the returned string is owned by Wren. You can inspect it
// while in your foreign method, but cannot keep a pointer to it after the
// function returns, since the garbage collector may reclaim it.
//
// Returns a pointer to the first byte of the array and fill [length] with the
// number of bytes in the array.
//
// It is an error to call this if the slot does not contain a string.
WREN_API const char* wrenGetSlotBytes(WrenVM* vm, int slot, int* length);
// Reads a number from [slot].
//
// It is an error to call this if the slot does not contain a number.
WREN_API double wrenGetSlotDouble(WrenVM* vm, int slot);
// Reads a foreign object from [slot] and returns a pointer to the foreign data
// stored with it.
//
// It is an error to call this if the slot does not contain an instance of a
// foreign class.
WREN_API void* wrenGetSlotForeign(WrenVM* vm, int slot);
// Reads a string from [slot].
//
// The memory for the returned string is owned by Wren. You can inspect it
// while in your foreign method, but cannot keep a pointer to it after the
// function returns, since the garbage collector may reclaim it.
//
// It is an error to call this if the slot does not contain a string.
WREN_API const char* wrenGetSlotString(WrenVM* vm, int slot);
// Creates a handle for the value stored in [slot].
//
// This will prevent the object that is referred to from being garbage collected
// until the handle is released by calling [wrenReleaseHandle()].
WREN_API WrenHandle* wrenGetSlotHandle(WrenVM* vm, int slot);
// Stores the boolean [value] in [slot].
WREN_API void wrenSetSlotBool(WrenVM* vm, int slot, bool value);
// Stores the array [length] of [bytes] in [slot].
//
// The bytes are copied to a new string within Wren's heap, so you can free
// memory used by them after this is called.
WREN_API void wrenSetSlotBytes(WrenVM* vm, int slot, const char* bytes, size_t length);
// Stores the numeric [value] in [slot].
WREN_API void wrenSetSlotDouble(WrenVM* vm, int slot, double value);
// Creates a new instance of the foreign class stored in [classSlot] with [size]
// bytes of raw storage and places the resulting object in [slot].
//
// This does not invoke the foreign class's constructor on the new instance. If
// you need that to happen, call the constructor from Wren, which will then
// call the allocator foreign method. In there, call this to create the object
// and then the constructor will be invoked when the allocator returns.
//
// Returns a pointer to the foreign object's data.
WREN_API void* wrenSetSlotNewForeign(WrenVM* vm, int slot, int classSlot, size_t size);
// Stores a new empty list in [slot].
WREN_API void wrenSetSlotNewList(WrenVM* vm, int slot);
// Stores a new empty map in [slot].
WREN_API void wrenSetSlotNewMap(WrenVM* vm, int slot);
// Stores null in [slot].
WREN_API void wrenSetSlotNull(WrenVM* vm, int slot);
// Stores the string [text] in [slot].
//
// The [text] is copied to a new string within Wren's heap, so you can free
// memory used by it after this is called. The length is calculated using
// [strlen()]. If the string may contain any null bytes in the middle, then you
// should use [wrenSetSlotBytes()] instead.
WREN_API void wrenSetSlotString(WrenVM* vm, int slot, const char* text);
// Stores the value captured in [handle] in [slot].
//
// This does not release the handle for the value.
WREN_API void wrenSetSlotHandle(WrenVM* vm, int slot, WrenHandle* handle);
// Returns the number of elements in the list stored in [slot].
WREN_API int wrenGetListCount(WrenVM* vm, int slot);
// Reads element [index] from the list in [listSlot] and stores it in
// [elementSlot].
WREN_API void wrenGetListElement(WrenVM* vm, int listSlot, int index, int elementSlot);
// Sets the value stored at [index] in the list at [listSlot],
// to the value from [elementSlot].
WREN_API void wrenSetListElement(WrenVM* vm, int listSlot, int index, int elementSlot);
// Takes the value stored at [elementSlot] and inserts it into the list stored
// at [listSlot] at [index].
//
// As in Wren, negative indexes can be used to insert from the end. To append
// an element, use `-1` for the index.
WREN_API void wrenInsertInList(WrenVM* vm, int listSlot, int index, int elementSlot);
// Returns the number of entries in the map stored in [slot].
WREN_API int wrenGetMapCount(WrenVM* vm, int slot);
// Returns true if the key in [keySlot] is found in the map placed in [mapSlot].
WREN_API bool wrenGetMapContainsKey(WrenVM* vm, int mapSlot, int keySlot);
// Retrieves a value with the key in [keySlot] from the map in [mapSlot] and
// stores it in [valueSlot].
WREN_API void wrenGetMapValue(WrenVM* vm, int mapSlot, int keySlot, int valueSlot);
// Takes the value stored at [valueSlot] and inserts it into the map stored
// at [mapSlot] with key [keySlot].
WREN_API void wrenSetMapValue(WrenVM* vm, int mapSlot, int keySlot, int valueSlot);
// Removes a value from the map in [mapSlot], with the key from [keySlot],
// and place it in [removedValueSlot]. If not found, [removedValueSlot] is
// set to null, the same behaviour as the Wren Map API.
WREN_API void wrenRemoveMapValue(WrenVM* vm, int mapSlot, int keySlot, int removedValueSlot);
// Looks up the top level variable with [name] in resolved [module] and stores
// it in [slot].
WREN_API void wrenGetVariable(WrenVM* vm, const char* module, const char* name, int slot);
// Looks up the top level variable with [name] in resolved [module],
// returns false if not found. The module must be imported at the time,
// use wrenHasModule to ensure that before calling.
WREN_API bool wrenHasVariable(WrenVM* vm, const char* module, const char* name);
// Returns true if [module] has been imported/resolved before, false if not.
WREN_API bool wrenHasModule(WrenVM* vm, const char* module);
// Sets the current fiber to be aborted, and uses the value in [slot] as the
// runtime error object.
WREN_API void wrenAbortFiber(WrenVM* vm, int slot);
// Returns the user data associated with the WrenVM.
WREN_API void* wrenGetUserData(WrenVM* vm);
// Sets user data associated with the WrenVM.
WREN_API void wrenSetUserData(WrenVM* vm, void* userData);
#endif

11
be/src/thirdparty/wren/wren.hpp vendored Normal file
View File

@ -0,0 +1,11 @@
#ifndef wren_hpp
#define wren_hpp
// This is a convenience header for users that want to compile Wren as C and
// link to it from a C++ application.
extern "C" {
#include "wren.h"
}
#endif

View File

@ -0,0 +1,44 @@
#pragma once
#include <string>
#include <wren.hpp>
#include "index.hpp"
#include "pop.hpp"
#include "push.hpp"
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
void setNextError(WrenVM* vm, std::string str);
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename T, typename... Args>
struct ForeignKlassAllocator {
static T* ctor(Args&&... args) { return new T(std::forward<Args>(args)...); }
template <size_t... Is>
static T* ctorFrom(WrenVM* vm, detail::index_list<Is...>) {
return ctor(PopHelper<Args>::f(vm, Is + 1)...);
}
static void allocate(WrenVM* vm) {
auto* memory = wrenSetSlotNewForeign(vm, 0, 0, sizeof(ForeignObject<T>));
new (memory) ForeignObject<T>();
auto* wrapper = reinterpret_cast<ForeignObject<T>*>(memory);
try {
wrapper->ptr.reset(ctorFrom(vm, detail::index_range<0, sizeof...(Args)>()));
} catch (std::exception& e) {
wrenEnsureSlots(vm, 1);
wrenSetSlotString(vm, 0, e.what());
wrenAbortFiber(vm, 0);
}
}
static void finalize(void* memory) {
auto* wrapper = reinterpret_cast<ForeignObject<T>*>(memory);
wrapper->~ForeignObject<T>();
}
};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,234 @@
#pragma once
#include <memory>
#include <typeinfo>
#include "handle.hpp"
#include "pop.hpp"
#include "push.hpp"
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
/**
* @ingroup wrenbind17
* @brief A return value when calling a Wren function (alias Any)
* @see Any
* @details This extends the lifetime of the Wren object (handle). As long as
* this ReturnValue instance exists the Wren object will exist.
* @note This variable can safely outlive the wrenbind17::VM class. If that happens
* then functions of this class will throw wrenbind17::RuntimeError exception. This
* holder will not try to free the Wren variable if the VM has been terminated. You
* don't have to worry about the lifetime of this holder. (uses weak pointers).
*/
class ReturnValue {
public:
ReturnValue() = default;
explicit ReturnValue(const WrenType type, Handle handle) : type(type), handle(std::move(handle)) {}
~ReturnValue() = default;
ReturnValue(const ReturnValue& other) = delete;
ReturnValue(ReturnValue&& other) noexcept { swap(other); }
ReturnValue& operator=(const ReturnValue& other) = delete;
ReturnValue& operator=(ReturnValue&& other) noexcept {
if (this != &other) {
swap(other);
}
return *this;
}
void swap(ReturnValue& other) noexcept {
std::swap(type, other.type);
std::swap(handle, other.handle);
}
/*!
* Returns the handle that this instance owns
*/
const Handle& getHandle() const { return handle; }
/*!
* Returns the handle that this instance owns
*/
Handle& getHandle() { return handle; }
/*!
* @brief The raw wren type held by this instance
*/
WrenType getType() const { return type; }
/*!
* @brief Check if the value held is some specific C++ type
* @note If the value held is a Wren numeric type then checking for any
* C++ integral or floating type will return true.
*/
template <class T>
bool is() const {
if (type == WREN_TYPE_NULL) {
return false;
}
if (const auto vm = handle.getVmWeak().lock()) {
wrenEnsureSlots(vm.get(), 1);
wrenSetSlotHandle(vm.get(), 0, handle.getHandle());
using Type = typename std::remove_reference<typename std::remove_pointer<T>::type>::type;
return detail::is<Type>(vm.get(), 0);
} else {
throw RuntimeError("Invalid handle");
}
}
bool isMap() const { return type == WREN_TYPE_MAP; }
bool isList() const { return type == WREN_TYPE_LIST; }
/*!
* @brief Returns the value
* @note If the value held is a Wren numeric type then getting for any
* C++ integral or floating type will result in a cast from a double to that type.
* @throws RuntimeError if this instance is invalid (constructed via the default constructor)
* @throws BadCast if the type required by specifying the template argument does not match the type held
* @tparam T the type you want to get
* @see shared()
*/
template <class T>
T as() {
if (type == WREN_TYPE_NULL) {
throw BadCast("Bad cast when getting value from Wren");
}
if (const auto vm = handle.getVmWeak().lock()) {
wrenEnsureSlots(vm.get(), 1);
wrenSetSlotHandle(vm.get(), 0, handle.getHandle());
return detail::PopHelper<T>::f(vm.get(), 0);
} else {
throw RuntimeError("Invalid handle");
}
}
/*!
* @brief Returns the value as a shared pointer
* @note Only works for custom C++ classes (foreign classes) that have been bound to the VM.
* @throws RuntimeError if this instance is invalid (constructed via the default constructor)
* @throws BadCast if the type required by specifying the template argument does not match the type held
* @tparam T the type of the std::shared_ptr you want to get
* @see as()
*/
template <class T>
std::shared_ptr<T> shared() {
return as<std::shared_ptr<T>>();
}
private:
WrenType type = WrenType::WREN_TYPE_NULL;
Handle handle;
};
#ifndef DOXYGEN_SHOULD_SKIP_THIS
template <>
inline bool ReturnValue::is<int8_t>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<char>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<short>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<int>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<long>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<long long>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<unsigned char>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<unsigned short>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<unsigned int>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<unsigned long>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<unsigned long long>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<float>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<double>() const {
return type == WREN_TYPE_NUM;
}
template <>
inline bool ReturnValue::is<bool>() const {
return type == WREN_TYPE_BOOL;
}
template <>
inline bool ReturnValue::is<std::nullptr_t>() const {
return type == WREN_TYPE_NULL;
}
template <>
inline bool ReturnValue::is<std::string>() const {
return type == WREN_TYPE_STRING;
}
template <>
inline std::nullptr_t ReturnValue::as<std::nullptr_t>() {
if (!is<std::nullptr_t>()) {
throw BadCast("Return value is not a null");
}
return nullptr;
}
#endif
/**
* @ingroup wrenbind17
* @see ReturnValue
* @brief An alias of ReturnValue class
*/
using Any = ReturnValue;
#ifndef DOXYGEN_SHOULD_SKIP_THIS
template <>
inline Any detail::getSlot(WrenVM* vm, const int idx) {
const auto type = wrenGetSlotType(vm, 0);
if (type == WREN_TYPE_NULL) {
return Any();
}
return Any(type, Handle(getSharedVm(vm), wrenGetSlotHandle(vm, idx)));
}
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,246 @@
#pragma once
#include <memory>
#include <wren.hpp>
#include "index.hpp"
#include "pop.hpp"
#include "push.hpp"
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename R>
struct ForeginMethodReturnHelper {
static inline void push(WrenVM* vm, int index, R ret) { PushHelper<R>::f(vm, index, ret); }
};
template <typename R>
struct ForeginMethodReturnHelper<R&> {
static inline void push(WrenVM* vm, int index, R& ret) { PushHelper<R*>::f(vm, index, &ret); }
};
template <typename R>
struct ForeginMethodReturnHelper<const R&> {
static inline void push(WrenVM* vm, int index, const R& ret) { PushHelper<R*>::f(vm, index, &const_cast<R&>(ret)); }
};
template <typename R>
struct ForeginMethodReturnHelper<R*> {
static inline void push(WrenVM* vm, int index, R* ret) { PushHelper<R*>::f(vm, index, ret); }
};
template <typename R>
struct ForeginMethodReturnHelper<const R*> {
static inline void push(WrenVM* vm, int index, const R* ret) { PushHelper<const R*>::f(vm, index, ret); }
};
template <typename R>
struct ForeginMethodReturnHelper<R&&> {
static inline void push(WrenVM* vm, int index, R&& ret) { PushHelper<R>::f(vm, index, std::move(ret)); }
};
template <typename R>
struct ForeginMethodReturnHelper<const std::shared_ptr<R>&> {
static inline void push(WrenVM* vm, int index, const std::shared_ptr<R>& ret) {
PushHelper<const std::shared_ptr<R>&>::f(vm, index, ret);
}
};
template <typename R>
struct ForeginMethodReturnHelper<std::shared_ptr<R>&> {
static inline void push(WrenVM* vm, int index, std::shared_ptr<R>& ret) {
PushHelper<std::shared_ptr<R>&>::f(vm, index, ret);
}
};
template <typename R>
struct ForeginMethodReturnHelper<const std::variant<R>&> {
static inline void push(WrenVM* vm, int index, const std::variant<R>& ret) {
PushHelper<const std::variant<R>&>::f(vm, index, ret);
}
};
template <typename R>
struct ForeginMethodReturnHelper<std::variant<R>&> {
static inline void push(WrenVM* vm, int index, std::variant<R>& ret) {
PushHelper<std::variant<R>&>::f(vm, index, ret);
}
};
template <>
inline void ForeginMethodReturnHelper<const std::string&>::push(WrenVM* vm, int index, const std::string& ret) {
PushHelper<const std::string&>::f(vm, index, ret);
}
template <>
inline void ForeginMethodReturnHelper<std::string&>::push(WrenVM* vm, int index, std::string& ret) {
PushHelper<std::string&>::f(vm, index, ret);
}
template <typename R, typename T, typename... Args>
struct ForeignMethodCaller {
template <R (T::*Fn)(Args...), size_t... Is>
static void callFrom(WrenVM* vm, detail::index_list<Is...>) {
auto self = PopHelper<T*>::f(vm, 0);
// R ret = (self->*Fn)(PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...);
// PushHelper<R>::f(vm, 0, ret);
ForeginMethodReturnHelper<R>::push(
vm, 0, (self->*Fn)(PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...));
}
template <R (T::*Fn)(Args...)>
static void call(WrenVM* vm) {
try {
callFrom<Fn>(vm, detail::index_range<0, sizeof...(Args)>());
} catch (...) {
exceptionHandler(vm, std::current_exception());
}
}
template <R (T::*Fn)(Args...) const, size_t... Is>
static void callFrom(WrenVM* vm, detail::index_list<Is...>) {
auto self = PopHelper<T*>::f(vm, 0);
// R ret = (self->*Fn)(PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...);
// PushHelper<R>::f(vm, 0, ret);
ForeginMethodReturnHelper<R>::push(
vm, 0, (self->*Fn)(PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...));
}
template <R (T::*Fn)(Args...) const>
static void call(WrenVM* vm) {
try {
callFrom<Fn>(vm, detail::index_range<0, sizeof...(Args)>());
} catch (...) {
exceptionHandler(vm, std::current_exception());
}
}
};
template <typename T, typename... Args>
struct ForeignMethodCaller<void, T, Args...> {
template <void (T::*Fn)(Args...), size_t... Is>
static void callFrom(WrenVM* vm, detail::index_list<Is...>) {
auto self = PopHelper<T*>::f(vm, 0);
(self->*Fn)(PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...);
}
template <void (T::*Fn)(Args...)>
static void call(WrenVM* vm) {
try {
callFrom<Fn>(vm, detail::index_range<0, sizeof...(Args)>());
} catch (...) {
exceptionHandler(vm, std::current_exception());
}
}
template <void (T::*Fn)(Args...) const, size_t... Is>
static void callFrom(WrenVM* vm, detail::index_list<Is...>) {
auto self = PopHelper<T*>::f(vm, 0);
(self->*Fn)(PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...);
}
template <void (T::*Fn)(Args...) const>
static void call(WrenVM* vm) {
try {
callFrom<Fn>(vm, detail::index_range<0, sizeof...(Args)>());
} catch (...) {
exceptionHandler(vm, std::current_exception());
}
}
};
template <typename R, typename T, typename... Args>
struct ForeignMethodExtCaller {
template <R (*Fn)(T&, Args...), size_t... Is>
static void callFrom(WrenVM* vm, detail::index_list<Is...>) {
auto self = PopHelper<T*>::f(vm, 0);
// R ret = (*Fn)(*self, PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...);
// PushHelper<R>::f(vm, 0, ret);
ForeginMethodReturnHelper<R>::push(
vm, 0, (*Fn)(*self, PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...));
}
template <R (*Fn)(T&, Args...)>
static void call(WrenVM* vm) {
try {
callFrom<Fn>(vm, detail::index_range<0, sizeof...(Args)>());
} catch (...) {
exceptionHandler(vm, std::current_exception());
}
}
};
template <typename T, typename... Args>
struct ForeignMethodExtCaller<void, T, Args...> {
template <void (*Fn)(T&, Args...), size_t... Is>
static void callFrom(WrenVM* vm, detail::index_list<Is...>) {
auto self = PopHelper<T*>::f(vm, 0);
(*Fn)(*self, PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...);
}
template <void (*Fn)(T&, Args...)>
static void call(WrenVM* vm) {
try {
callFrom<Fn>(vm, detail::index_range<0, sizeof...(Args)>());
} catch (...) {
exceptionHandler(vm, std::current_exception());
}
}
};
template <typename R, typename... Args>
struct ForeignFunctionCaller {
template <R (*Fn)(Args...), size_t... Is>
static void callFrom(WrenVM* vm, detail::index_list<Is...>) {
// R ret = (*Fn)(PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...);
// PushHelper<R>::f(vm, 0, ret);
ForeginMethodReturnHelper<R>::push(vm, 0,
(*Fn)(PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...));
}
template <R (*Fn)(Args...)>
static void call(WrenVM* vm) {
try {
callFrom<Fn>(vm, detail::index_range<0, sizeof...(Args)>());
} catch (...) {
exceptionHandler(vm, std::current_exception());
}
}
};
template <typename... Args>
struct ForeignFunctionCaller<void, Args...> {
template <void (*Fn)(Args...), size_t... Is>
static void callFrom(WrenVM* vm, detail::index_list<Is...>) {
(*Fn)(PopHelper<typename std::remove_const<Args>::type>::f(vm, Is + 1)...);
}
template <void (*Fn)(Args...)>
static void call(WrenVM* vm) {
try {
callFrom<Fn>(vm, detail::index_range<0, sizeof...(Args)>());
} catch (...) {
exceptionHandler(vm, std::current_exception());
}
}
};
template <typename T, typename V, V T::*Ptr>
struct ForeignPropCaller {
static void setter(WrenVM* vm) {
auto self = PopHelper<T*>::f(vm, 0);
self->*Ptr = PopHelper<V>::f(vm, 1);
}
static void getter(WrenVM* vm) {
auto self = PopHelper<T*>::f(vm, 0);
PushHelper<V>::f(vm, 0, std::forward<decltype(self->*Ptr)>(self->*Ptr));
}
};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,59 @@
#pragma once
#include <memory>
#include <stdexcept>
#include <string>
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
/**
* @ingroup wrenbind17
*/
class Exception : public std::exception {
public:
Exception() = default;
explicit Exception(std::string msg) : msg(std::move(msg)) {}
const char* what() const throw() override { return msg.c_str(); }
private:
std::string msg;
};
/**
* @ingroup wrenbind17
*/
class NotFound : public Exception {
public:
NotFound() : Exception("Not found") {}
};
/**
* @ingroup wrenbind17
*/
class BadCast : public Exception {
public:
BadCast() : Exception("Bad cast") {}
explicit BadCast(std::string msg) : Exception(std::move(msg)) {}
};
/**
* @ingroup wrenbind17
*/
class RuntimeError : public Exception {
public:
explicit RuntimeError(std::string msg) : Exception(std::move(msg)) {}
};
/**
* @ingroup wrenbind17
*/
class CompileError : public Exception {
public:
explicit CompileError(std::string msg) : Exception(std::move(msg)) {}
};
} // namespace wrenbind17

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,64 @@
#pragma once
#include <memory>
#include <wren.hpp>
#include "exception.hpp"
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
std::shared_ptr<WrenVM> getSharedVm(WrenVM* vm);
/**
* @ingroup wrenbind17
* @brief Holds a reference to some Wren type
* @details This is used by Map, Method, and Variable classes.
*/
class Handle {
public:
Handle() : handle(nullptr) {}
Handle(const std::shared_ptr<WrenVM> vm, WrenHandle* handle) : vm(vm), handle(handle) {}
~Handle() { reset(); }
Handle(const Handle& other) = delete;
Handle(Handle&& other) noexcept : handle(nullptr) { swap(other); }
Handle& operator=(const Handle& other) = delete;
Handle& operator=(Handle&& other) noexcept {
if (this != &other) {
swap(other);
}
return *this;
}
void swap(Handle& other) noexcept {
std::swap(vm, other.vm);
std::swap(handle, other.handle);
}
WrenHandle* getHandle() const { return handle; }
WrenVM* getVm() const {
if (const auto ptr = vm.lock()) {
return ptr.get();
} else {
throw RuntimeError("Invalid handle");
}
}
const std::weak_ptr<WrenVM>& getVmWeak() const { return vm; }
void reset() {
if (!vm.expired() && handle) {
wrenReleaseHandle(vm.lock().get(), handle);
vm.reset();
handle = nullptr;
}
}
operator bool() const { return !vm.expired() && handle; }
private:
std::weak_ptr<WrenVM> vm;
WrenHandle* handle;
};
} // namespace wrenbind17

View File

@ -0,0 +1,33 @@
#pragma once
#include <memory>
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <size_t... Is>
struct index_list {};
// Declare primary template for index range builder
template <size_t MIN, size_t N, size_t... Is>
struct range_builder;
// Base step
template <size_t MIN, size_t... Is>
struct range_builder<MIN, MIN, Is...> {
typedef index_list<Is...> type;
};
// Induction step
template <size_t MIN, size_t N, size_t... Is>
struct range_builder : public range_builder<MIN, N - 1, N - 1, Is...> {};
// Meta-function that returns a [MIN, MAX) index range
template <size_t MIN, size_t MAX>
using index_range = typename detail::range_builder<MIN, MAX>::type;
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,150 @@
#pragma once
#include "method.hpp"
#include "pop.hpp"
#include "push.hpp"
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
/**
* @ingroup wrenbind17
* @brief Holds native Wren map
*/
class Map {
public:
Map() {}
Map(const std::shared_ptr<Handle>& handle) : handle(handle) {}
~Map() { reset(); }
Handle& getHandle() { return *handle; }
const Handle& getHandle() const { return *handle; }
operator bool() const { return handle.operator bool(); }
void reset() { handle.reset(); }
/*!
* @brief Checks if a key exists in this map
* @throws RuntimeError if this is an invalid map or the Wren VM has terminated
* @warning If using strings, make sure that you use std::string because
* raw C-strings are not allowed.
* @note This function accepts any type you want. Integers, strings, booleans,
* does not matter as long as Wren map supports that key type.
*/
template <typename Key>
bool contains(const Key& key) const {
if (const auto ptr = handle->getVmWeak().lock()) {
wrenEnsureSlots(ptr.get(), 2);
wrenSetSlotHandle(ptr.get(), 0, handle->getHandle());
detail::PushHelper<Key>::f(ptr.get(), 1, key);
return wrenGetMapContainsKey(ptr.get(), 0, 1);
} else {
throw RuntimeError("Invalid handle");
}
}
/*!
* @brief Returns a value specified by T from the map by a key
* @throws NotFound if the key does not exist in the map
* @throws RuntimeError if this is an invalid map or the Wren VM has terminated
* @warning If using strings, make sure that you use std::string because
* raw C-strings are not allowed.
* @note This function accepts any type you want. Integers, strings, booleans,
* does not matter as long as Wren map supports that key type.
*/
template <typename T, typename Key>
T get(const Key& key) const {
if (const auto ptr = handle->getVmWeak().lock()) {
wrenEnsureSlots(ptr.get(), 3);
wrenSetSlotHandle(ptr.get(), 0, handle->getHandle());
detail::PushHelper<Key>::f(ptr.get(), 1, key);
if (!wrenGetMapContainsKey(ptr.get(), 0, 1)) {
throw NotFound();
}
wrenGetMapValue(ptr.get(), 0, 1, 2);
return detail::PopHelper<T>::f(ptr.get(), 2);
} else {
throw RuntimeError("Invalid handle");
}
}
/*!
* @brief Erases a key from the map
* @throws NotFound if the key does not exist in the map
* @throws RuntimeError if this is an invalid map or the Wren VM has terminated
* @returns true if the key has been erased, otherwise false
* @warning If using strings, make sure that you use std::string because
* raw C-strings are not allowed.
* @note This function accepts any type you want. Integers, strings, booleans,
* does not matter as long as Wren map supports that key type.
*/
template <typename Key>
bool erase(const Key& key) const {
if (const auto ptr = handle->getVmWeak().lock()) {
wrenEnsureSlots(ptr.get(), 3);
wrenSetSlotHandle(ptr.get(), 0, handle->getHandle());
detail::PushHelper<Key>::f(ptr.get(), 1, key);
wrenRemoveMapValue(ptr.get(), 0, 1, 2);
return !detail::is<std::nullptr_t>(ptr.get(), 2);
} else {
throw RuntimeError("Invalid handle");
}
}
/*!
* @brief Returns the size of the map
* @throws RuntimeError if this is an invalid map or the Wren VM has terminated
*/
size_t count() const {
if (const auto ptr = handle->getVmWeak().lock()) {
wrenEnsureSlots(ptr.get(), 1);
wrenSetSlotHandle(ptr.get(), 0, handle->getHandle());
return wrenGetMapCount(ptr.get(), 0);
} else {
throw RuntimeError("Invalid handle");
}
}
private:
std::shared_ptr<Handle> handle;
};
template <>
inline Map detail::getSlot<Map>(WrenVM* vm, const int idx) {
validate<WrenType::WREN_TYPE_MAP>(vm, idx);
return Map(std::make_shared<Handle>(getSharedVm(vm), wrenGetSlotHandle(vm, idx)));
}
template <>
inline bool detail::is<Map>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WREN_TYPE_MAP;
}
template <>
inline void detail::PushHelper<Map>::f(WrenVM* vm, int idx, const Map& value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
template <>
inline void detail::PushHelper<Map>::f(WrenVM* vm, int idx, Map&& value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
template <>
inline void detail::PushHelper<const Map>::f(WrenVM* vm, int idx, const Map value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
template <>
inline void detail::PushHelper<const Map&>::f(WrenVM* vm, int idx, const Map& value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
template <>
inline void detail::PushHelper<Map&>::f(WrenVM* vm, int idx, Map& value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
} // namespace wrenbind17

View File

@ -0,0 +1,78 @@
#pragma once
#include <memory>
#include <wren.hpp>
#include "any.hpp"
#include "exception.hpp"
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
inline void pushArgs(WrenVM* vm, int idx) {
(void)vm;
(void)idx;
}
template <typename First, typename... Other>
inline void pushArgs(WrenVM* vm, int idx, First&& first, Other&&... other) {
PushHelper<First>::f(vm, idx, std::forward<First>(first));
pushArgs(vm, ++idx, std::forward<Other>(other)...);
}
template <typename... Args>
struct CallAndReturn {
static Any func(WrenVM* vm, WrenHandle* handle, WrenHandle* func, Args&&... args) {
constexpr auto n = sizeof...(Args);
wrenEnsureSlots(vm, n + 1);
wrenSetSlotHandle(vm, 0, handle);
pushArgs(vm, 1, std::forward<Args>(args)...);
if (wrenCall(vm, func) != WREN_RESULT_SUCCESS) {
throw RuntimeError(getLastError(vm));
}
return getSlot<Any>(vm, 0);
}
};
} // namespace detail
#endif
/**
* @ingroup wrenbind17
*/
class Method {
public:
Method() = default;
Method(std::shared_ptr<Handle> variable, std::shared_ptr<Handle> handle)
: variable(std::move(variable)), handle(std::move(handle)) {}
~Method() { reset(); }
template <typename... Args>
Any operator()(Args&&... args) {
if (const auto ptr = handle->getVmWeak().lock().get()) {
return detail::CallAndReturn<Args...>::func(ptr, variable->getHandle(), handle->getHandle(),
std::forward<Args>(args)...);
} else {
throw RuntimeError("Invalid handle");
}
}
operator bool() const { return variable && handle; }
void reset() {
handle.reset();
variable.reset();
}
private:
std::shared_ptr<Handle> variable;
std::shared_ptr<Handle> handle;
};
} // namespace wrenbind17

View File

@ -0,0 +1,89 @@
#pragma once
#include <sstream>
#include <string>
#include <unordered_map>
#include <vector>
#include <wren.hpp>
#include "foreign.hpp"
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
void addClassType(WrenVM* vm, const std::string& module, const std::string& name, size_t hash);
void addClassCast(WrenVM* vm, std::shared_ptr<detail::ForeignPtrConvertor> convertor, size_t hash, size_t other);
/**
* @ingroup wrenbind17
*/
class ForeignModule {
public:
ForeignModule(std::string name, WrenVM* vm) : name(std::move(name)), vm(vm) {}
ForeignModule(const ForeignModule& other) = delete;
ForeignModule(ForeignModule&& other) noexcept : vm(nullptr) { swap(other); }
~ForeignModule() = default;
ForeignModule& operator=(const ForeignModule& other) = delete;
ForeignModule& operator=(ForeignModule&& other) noexcept {
if (this != &other) {
swap(other);
}
return *this;
}
void swap(ForeignModule& other) {
std::swap(klasses, other.klasses);
std::swap(vm, other.vm);
std::swap(name, other.name);
}
template <typename T, typename... Others>
ForeignKlassImpl<T>& klass(std::string name) {
insertKlassCast<T, Others...>();
auto ptr = std::make_unique<ForeignKlassImpl<T>>(std::move(name));
auto ret = ptr.get();
addClassType(vm, this->name, ptr->getName(), typeid(T).hash_code());
klasses.insert(std::make_pair(ptr->getName(), std::move(ptr)));
return *ret;
}
std::string str() const {
std::stringstream ss;
for (const auto& pair : klasses) {
pair.second->generate(ss);
}
for (const auto& r : raw) {
ss << r << "\n";
}
return ss.str();
}
void append(std::string text) { raw.push_back(std::move(text)); }
ForeignKlass& findKlass(const std::string& name) {
auto it = klasses.find(name);
if (it == klasses.end()) throw NotFound();
return *it->second;
}
const std::string& getName() const { return name; }
private:
template <typename T>
void insertKlassCast() {
// void
}
template <typename T, typename Other, typename... Others>
void insertKlassCast() {
addClassCast(vm, std::make_shared<detail::ForeignObjectSharedPtrConvertor<T, Other>>(), typeid(T).hash_code(),
typeid(Other).hash_code());
insertKlassCast<T, Others...>();
}
std::string name;
WrenVM* vm;
std::unordered_map<std::string, std::unique_ptr<ForeignKlass>> klasses;
std::vector<std::string> raw;
};
} // namespace wrenbind17

View File

@ -0,0 +1,105 @@
#pragma once
#include <cstdlib>
#include <memory>
#include <string>
#include <typeinfo>
#include <variant>
#include <wren.hpp>
#include "exception.hpp"
#include "handle.hpp"
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
std::string getLastError(WrenVM* vm);
inline void exceptionHandler(WrenVM* vm, const std::exception_ptr& eptr) {
try {
if (eptr) {
std::rethrow_exception(eptr);
} else {
wrenEnsureSlots(vm, 1);
wrenSetSlotString(vm, 0, "Unknown error");
wrenAbortFiber(vm, 0);
}
} catch (std::exception& e) {
wrenEnsureSlots(vm, 1);
wrenSetSlotString(vm, 0, e.what());
wrenAbortFiber(vm, 0);
}
}
template <class T>
struct is_shared_ptr : std::false_type {};
template <class T>
struct is_shared_ptr<std::shared_ptr<T>> : std::true_type {};
namespace detail {
class Foreign {
public:
Foreign() = default;
virtual ~Foreign() = 0;
virtual void* get() const = 0;
virtual size_t hash() const = 0;
};
inline Foreign::~Foreign() {}
template <typename T>
class ForeignObject : public Foreign {
public:
ForeignObject() {}
ForeignObject(std::shared_ptr<T> ptr) : ptr(std::move(ptr)) {}
virtual ~ForeignObject() = default;
void* get() const override { return ptr.get(); }
size_t hash() const override { return typeid(T).hash_code(); }
const std::shared_ptr<T>& shared() const { return ptr; }
std::shared_ptr<T> ptr;
};
class ForeignPtrConvertor {
public:
ForeignPtrConvertor() = default;
virtual ~ForeignPtrConvertor() = default;
};
template <typename T>
class ForeignSharedPtrConvertor : public ForeignPtrConvertor {
public:
ForeignSharedPtrConvertor() = default;
virtual ~ForeignSharedPtrConvertor() = default;
virtual std::shared_ptr<T> cast(Foreign* foreign) const = 0;
};
template <typename From, typename To>
class ForeignObjectSharedPtrConvertor : public ForeignSharedPtrConvertor<To> {
public:
ForeignObjectSharedPtrConvertor() = default;
virtual ~ForeignObjectSharedPtrConvertor() = default;
inline std::shared_ptr<To> cast(Foreign* foreign) const override {
if (!foreign) throw Exception("Cannot upcast foreign pointer is null and this should not happen");
auto* ptr = dynamic_cast<ForeignObject<From>*>(foreign);
if (!ptr) throw BadCast("Bad cast while upcasting to a base type");
return std::dynamic_pointer_cast<To>(ptr->shared());
}
};
template <class T>
struct is_shared_ptr : std::false_type {};
template <class T>
struct is_shared_ptr<std::shared_ptr<T>> : std::true_type {};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,389 @@
#pragma once
#include <memory>
#include <string>
#include <wren.hpp>
#include "object.hpp"
namespace wrenbind17 {
void getClassType(WrenVM* vm, std::string& module, std::string& name, size_t hash);
detail::ForeignPtrConvertor* getClassCast(WrenVM* vm, size_t hash, size_t other);
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
// ============================================================================================================
// CHECK SLOTS FOR TYPE
// ============================================================================================================
inline const char* wrenSlotTypeToStr(const WrenType type) {
switch (type) {
case WREN_TYPE_BOOL:
return "bool";
case WREN_TYPE_FOREIGN:
return "instance";
case WREN_TYPE_LIST:
return "list";
case WREN_TYPE_NULL:
return "null";
case WREN_TYPE_NUM:
return "number";
case WREN_TYPE_STRING:
return "string";
case WREN_TYPE_UNKNOWN:
default:
return "unknown";
}
}
template <typename T>
inline bool is(WrenVM* vm, const int idx) {
const auto type = wrenGetSlotType(vm, idx);
if (type != WrenType::WREN_TYPE_FOREIGN) return false;
auto slot = wrenGetSlotForeign(vm, idx);
const auto foreign = reinterpret_cast<Foreign*>(slot);
return foreign->hash() == typeid(T).hash_code();
}
template <>
inline bool is<bool>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_BOOL;
}
template <>
inline bool is<std::string>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_STRING;
}
template <>
inline bool is<std::nullptr_t>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NULL;
}
template <>
inline bool is<float>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<double>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<int>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<int8_t>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<char>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<unsigned char>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<short>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<unsigned short>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<unsigned>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<long>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<long long>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<unsigned long>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <>
inline bool is<unsigned long long>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WrenType::WREN_TYPE_NUM;
}
template <typename T>
struct CheckSlot {
static bool f(WrenVM* vm, const int idx) { return is<T>(vm, idx); }
};
template <typename T>
struct CheckSlot<T&> {
static bool f(WrenVM* vm, const int idx) { return is<T>(vm, idx); }
};
template <typename T>
struct CheckSlot<const T&> {
static bool f(WrenVM* vm, const int idx) { return is<T>(vm, idx); }
};
template <typename T>
struct CheckSlot<T*> {
static bool f(WrenVM* vm, const int idx) { return is<T>(vm, idx); }
};
template <typename T>
struct CheckSlot<const T*> {
static bool f(WrenVM* vm, const int idx) { return is<T>(vm, idx); }
};
template <typename T>
struct CheckSlot<std::shared_ptr<T>> {
static bool f(WrenVM* vm, const int idx) { return is<T>(vm, idx); }
};
template <typename T>
struct CheckSlot<const std::shared_ptr<T>&> {
static bool f(WrenVM* vm, const int idx) { return is<T>(vm, idx); }
};
// ============================================================================================================
// BASIC TYPES
// ============================================================================================================
template <WrenType Type>
inline void validate(WrenVM* vm, int idx) {
const auto t = wrenGetSlotType(vm, idx);
if (t != Type)
throw BadCast("Bad cast when getting value from Wren got " + std::string(wrenSlotTypeToStr(t)) + " expected " +
std::string(wrenSlotTypeToStr(Type)));
}
template <typename T>
std::shared_ptr<T> getSlotForeign(WrenVM* vm, void* slot) {
using Type = typename std::remove_const<typename std::remove_pointer<T>::type>::type;
using ForeignTypeConvertor = ForeignSharedPtrConvertor<Type>;
const auto foreign = reinterpret_cast<Foreign*>(slot);
if (foreign->hash() != typeid(Type).hash_code()) {
try {
auto base = getClassCast(vm, foreign->hash(), typeid(Type).hash_code());
auto derived = reinterpret_cast<ForeignTypeConvertor*>(base);
if (!derived) {
throw BadCast("Bad cast the value cannot be upcast to the expected type");
}
return derived->cast(foreign);
} catch (std::out_of_range& e) {
(void)e;
throw BadCast("Bad cast the value is not the expected type");
}
}
auto ptr = reinterpret_cast<ForeignObject<Type>*>(foreign);
return ptr->shared();
}
template <typename T>
const std::shared_ptr<T> getSlotForeign(WrenVM* vm, const int idx) {
validate<WrenType::WREN_TYPE_FOREIGN>(vm, idx);
return getSlotForeign<T>(vm, wrenGetSlotForeign(vm, idx));
}
template <typename T>
T getSlot(WrenVM* vm, int idx) {
static_assert(!std::is_same<std::string, T>(), "type can't be std::string");
static_assert(!is_shared_ptr<T>::value, "type can't be shared_ptr<T>");
validate<WrenType::WREN_TYPE_FOREIGN>(vm, idx);
return *getSlotForeign<typename std::remove_reference<T>::type>(vm, idx).get();
}
template <typename T>
struct PopHelper {
static inline T f(WrenVM* vm, int idx) { return getSlot<T>(vm, idx); }
};
template <typename T>
struct PopHelper<T*> {
static inline T* f(WrenVM* vm, int idx) {
const auto type = wrenGetSlotType(vm, idx);
if (type == WrenType::WREN_TYPE_NULL)
return nullptr;
else if (type != WrenType::WREN_TYPE_FOREIGN)
throw BadCast("Bad cast when getting value from Wren");
return getSlotForeign<typename std::remove_const<T>::type>(vm, idx).get();
}
};
template <typename T>
struct PopHelper<const T&> {
static inline const T& f(WrenVM* vm, int idx) {
static_assert(!std::is_same<std::string, T>(), "type can't be std::string");
static_assert(!std::is_same<std::nullptr_t, T>(), "type can't be std::nullptr_t");
static_assert(!is_shared_ptr<T>::value, "type can't be shared_ptr<T>");
return *getSlotForeign<T>(vm, idx).get();
}
};
template <typename T>
struct PopHelper<std::shared_ptr<T>> {
static inline std::shared_ptr<T> f(WrenVM* vm, int idx) {
const auto type = wrenGetSlotType(vm, idx);
if (type == WrenType::WREN_TYPE_NULL)
return nullptr;
else if (type != WrenType::WREN_TYPE_FOREIGN)
throw BadCast("Bad cast when getting value from Wren");
return getSlotForeign<T>(vm, idx);
}
};
template <typename T>
struct PopHelper<const std::shared_ptr<T>&> {
static inline std::shared_ptr<T> f(WrenVM* vm, int idx) {
const auto type = wrenGetSlotType(vm, idx);
if (type == WrenType::WREN_TYPE_NULL)
return nullptr;
else if (type != WrenType::WREN_TYPE_FOREIGN)
throw BadCast("Bad cast when getting value from Wren");
return getSlotForeign<T>(vm, idx);
}
};
template <>
inline Handle getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_UNKNOWN>(vm, idx);
return Handle(getSharedVm(vm), wrenGetSlotHandle(vm, idx));
}
template <>
inline std::string getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_STRING>(vm, idx);
return std::string(wrenGetSlotString(vm, idx));
}
template <>
inline std::nullptr_t getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NULL>(vm, idx);
return nullptr;
}
template <>
inline bool getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_BOOL>(vm, idx);
return wrenGetSlotBool(vm, idx);
}
template <>
inline int8_t getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<int8_t>(wrenGetSlotDouble(vm, idx));
}
template <>
inline char getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<char>(wrenGetSlotDouble(vm, idx));
}
template <>
inline int getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<int>(wrenGetSlotDouble(vm, idx));
}
template <>
inline short getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<short>(wrenGetSlotDouble(vm, idx));
}
template <>
inline long getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<long>(wrenGetSlotDouble(vm, idx));
}
template <>
inline unsigned char getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<unsigned char>(wrenGetSlotDouble(vm, idx));
}
template <>
inline unsigned long getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<unsigned long>(wrenGetSlotDouble(vm, idx));
}
template <>
inline unsigned short getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<unsigned short>(wrenGetSlotDouble(vm, idx));
}
template <>
inline unsigned getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<unsigned>(wrenGetSlotDouble(vm, idx));
}
template <>
inline long long getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<long long>(wrenGetSlotDouble(vm, idx));
}
template <>
inline unsigned long long getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<unsigned long long>(wrenGetSlotDouble(vm, idx));
}
template <>
inline float getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<float>(wrenGetSlotDouble(vm, idx));
}
template <>
inline double getSlot(WrenVM* vm, int idx) {
validate<WrenType::WREN_TYPE_NUM>(vm, idx);
return static_cast<double>(wrenGetSlotDouble(vm, idx));
}
#define WRENBIND17_POP_HELPER(Type) \
template <> \
struct PopHelper<const Type&> { \
static inline Type f(WrenVM* vm, int idx) { return getSlot<Type>(vm, idx); } \
};
WRENBIND17_POP_HELPER(std::string)
WRENBIND17_POP_HELPER(std::nullptr_t)
WRENBIND17_POP_HELPER(bool)
WRENBIND17_POP_HELPER(int8_t)
WRENBIND17_POP_HELPER(char)
WRENBIND17_POP_HELPER(int)
WRENBIND17_POP_HELPER(short)
WRENBIND17_POP_HELPER(long)
WRENBIND17_POP_HELPER(unsigned long)
WRENBIND17_POP_HELPER(unsigned)
WRENBIND17_POP_HELPER(long long)
WRENBIND17_POP_HELPER(unsigned long long)
WRENBIND17_POP_HELPER(unsigned short)
WRENBIND17_POP_HELPER(unsigned char)
WRENBIND17_POP_HELPER(float)
WRENBIND17_POP_HELPER(double)
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,296 @@
#pragma once
#include <string>
#include <wren.hpp>
#include "exception.hpp"
#include "object.hpp"
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
void getClassType(WrenVM* vm, std::string& module, std::string& name, size_t hash);
bool isClassRegistered(WrenVM* vm, const size_t hash);
detail::ForeignPtrConvertor* getClassCast(WrenVM* vm, size_t hash, size_t other);
namespace detail {
template <typename T>
struct PushHelper;
template <typename T>
void pushAsConstRef(WrenVM* vm, int idx, const T& value) {
static_assert(!std::is_same<int, typename std::remove_const<T>::type>(), "type can't be int");
static_assert(!std::is_same<std::string, typename std::remove_const<T>::type>(), "type can't be std::string");
static_assert(!is_shared_ptr<T>::value, "type can't be shared_ptr<T>");
try {
std::string module;
std::string klass;
getClassType(vm, module, klass, typeid(T).hash_code());
wrenEnsureSlots(vm, idx + 1);
wrenGetVariable(vm, module.c_str(), klass.c_str(), idx);
auto memory = wrenSetSlotNewForeign(vm, idx, idx, sizeof(ForeignObject<T>));
auto* foreign = new (memory) ForeignObject<T>(std::make_shared<T>(value));
(void)foreign;
} catch (std::out_of_range& e) {
(void)e;
throw BadCast("Class type not registered in Wren VM");
}
}
template <typename T>
void pushAsMove(WrenVM* vm, int idx, T&& value) {
static_assert(!std::is_same<int, T>(), "type can't be int");
static_assert(!std::is_same<std::string, T>(), "type can't be std::string");
static_assert(!is_shared_ptr<T>::value, "type can't be shared_ptr<T>");
try {
std::string module;
std::string klass;
getClassType(vm, module, klass, typeid(T).hash_code());
wrenEnsureSlots(vm, idx + 1);
wrenGetVariable(vm, module.c_str(), klass.c_str(), idx);
auto memory = wrenSetSlotNewForeign(vm, idx, idx, sizeof(ForeignObject<T>));
auto* foreign = new (memory) ForeignObject<T>(std::make_shared<T>(std::move(value)));
(void)foreign;
} catch (std::out_of_range& e) {
(void)e;
throw BadCast("Class type not registered in Wren VM");
}
}
template <typename T>
void pushAsPtr(WrenVM* vm, int idx, T* value) {
static_assert(!std::is_same<int, T>(), "type can't be int");
static_assert(!std::is_same<std::string, T>(), "type can't be std::string");
static_assert(!is_shared_ptr<T>::value, "type can't be shared_ptr<T>");
try {
std::string module;
std::string klass;
getClassType(vm, module, klass, typeid(T).hash_code());
wrenEnsureSlots(vm, idx + 1);
wrenGetVariable(vm, module.c_str(), klass.c_str(), idx);
auto memory = wrenSetSlotNewForeign(vm, idx, idx, sizeof(ForeignObject<T>));
auto* foreign = new (memory) ForeignObject<T>(std::shared_ptr<T>(value, [](T* t) {}));
(void)foreign;
} catch (std::out_of_range& e) {
(void)e;
throw BadCast("Class type not registered in Wren VM");
}
}
template <typename T>
struct PushHelper {
static inline void f(WrenVM* vm, int idx, const T& value) { pushAsConstRef(vm, idx, value); }
static inline void f(WrenVM* vm, int idx, T&& value) { pushAsMove(vm, idx, std::move(value)); }
};
template <typename T>
struct PushHelper<const T> {
static inline void f(WrenVM* vm, int idx, const T value) { PushHelper<T*>::f(vm, idx, value); }
};
template <typename T>
struct PushHelper<T*> {
static inline void f(WrenVM* vm, int idx, T* value) { pushAsPtr(vm, idx, value); }
};
template <typename T>
struct PushHelper<const T*> {
static inline void f(WrenVM* vm, int idx, const T* value) { PushHelper<T*>::f(vm, idx, const_cast<T*>(value)); }
};
template <typename T>
struct PushHelper<const T*&> {
static inline void f(WrenVM* vm, int idx, const T*& value) { PushHelper<T*>::f(vm, idx, const_cast<T*>(value)); }
};
template <typename T>
struct PushHelper<T&> {
static inline void f(WrenVM* vm, int idx, T& value) { PushHelper<T>::f(vm, idx, static_cast<const T&>(value)); }
};
template <typename T>
struct PushHelper<const T&> {
static inline void f(WrenVM* vm, int idx, const T& value) {
PushHelper<T>::f(vm, idx, static_cast<const T&>(value));
}
};
// ============================================================================================================
// BASIC TYPES
// ============================================================================================================
#define WRENBIND17_PUSH_HELPER(T, FUNC) \
template <> \
inline void PushHelper<T>::f(WrenVM* vm, int idx, const T& value) { \
FUNC; \
} \
template <> \
inline void PushHelper<T>::f(WrenVM* vm, int idx, T&& value) { \
FUNC; \
} \
template <> \
inline void PushHelper<const T>::f(WrenVM* vm, int idx, const T value) { \
FUNC; \
} \
template <> \
inline void PushHelper<T&>::f(WrenVM* vm, int idx, T& value) { \
FUNC; \
} \
template <> \
inline void PushHelper<const T&>::f(WrenVM* vm, int idx, const T& value) { \
FUNC; \
} \
template <> \
inline void PushHelper<T*>::f(WrenVM* vm, int idx, T* value) { \
PushHelper<T>::f(vm, idx, *value); \
} \
template <> \
inline void PushHelper<const T*>::f(WrenVM* vm, int idx, const T* value) { \
PushHelper<T>::f(vm, idx, *value); \
}
WRENBIND17_PUSH_HELPER(char, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(int8_t, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(short, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(int, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(long, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(long long, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(uint8_t, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(unsigned short, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(unsigned int, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(unsigned long, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(unsigned long long, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(float, wrenSetSlotDouble(vm, idx, static_cast<double>(value)));
WRENBIND17_PUSH_HELPER(double, wrenSetSlotDouble(vm, idx, value));
WRENBIND17_PUSH_HELPER(bool, wrenSetSlotBool(vm, idx, value));
WRENBIND17_PUSH_HELPER(std::nullptr_t, wrenSetSlotNull(vm, idx));
template <>
struct PushHelper<std::string> {
static inline void f(WrenVM* vm, int idx, const std::string value) { wrenSetSlotString(vm, idx, value.c_str()); }
};
template <size_t N>
struct PushHelper<const char (&)[N]> {
static inline void f(WrenVM* vm, int idx, const char (&value)[N]) { wrenSetSlotString(vm, idx, value); }
};
template <size_t N>
struct PushHelper<char (&)[N]> {
static inline void f(WrenVM* vm, int idx, char (&value)[N]) { wrenSetSlotString(vm, idx, value); }
};
template <>
struct PushHelper<const char*&> {
static inline void f(WrenVM* vm, int idx, const char*& value) { wrenSetSlotString(vm, idx, value); }
};
template <>
struct PushHelper<char*&> {
static inline void f(WrenVM* vm, int idx, char*& value) { wrenSetSlotString(vm, idx, value); }
};
template <>
struct PushHelper<const std::string> {
static inline void f(WrenVM* vm, int idx, const std::string value) { wrenSetSlotString(vm, idx, value.c_str()); }
};
template <>
struct PushHelper<std::string&&> {
static inline void f(WrenVM* vm, int idx, std::string&& value) { wrenSetSlotString(vm, idx, value.c_str()); }
};
template <>
struct PushHelper<std::string&> {
static inline void f(WrenVM* vm, int idx, std::string& value) { wrenSetSlotString(vm, idx, value.c_str()); }
};
template <>
struct PushHelper<const std::string&> {
static inline void f(WrenVM* vm, int idx, const std::string& value) { wrenSetSlotString(vm, idx, value.c_str()); }
};
template <typename T>
struct PushHelper<std::shared_ptr<T>> {
static inline void f(WrenVM* vm, int idx, std::shared_ptr<T> value) {
static_assert(!std::is_same<std::string, T>(), "type can't be std::string");
static_assert(!is_shared_ptr<T>::value, "type can't be shared_ptr<T>");
try {
std::string module;
std::string klass;
getClassType(vm, module, klass, typeid(T).hash_code());
wrenEnsureSlots(vm, idx + 1);
wrenGetVariable(vm, module.c_str(), klass.c_str(), idx);
auto memory = wrenSetSlotNewForeign(vm, idx, idx, sizeof(ForeignObject<T>));
auto* foreign = new (memory) ForeignObject<T>(value);
(void)foreign;
} catch (std::out_of_range& e) {
(void)e;
throw BadCast("Class type not registered in Wren VM");
}
}
};
template <typename T>
struct PushHelper<const std::shared_ptr<T>> {
static inline void f(WrenVM* vm, int idx, const std::shared_ptr<T> value) {
PushHelper<std::shared_ptr<T>>::f(vm, idx, value);
}
};
template <typename T>
struct PushHelper<std::shared_ptr<T>&&> {
static inline void f(WrenVM* vm, int idx, std::shared_ptr<T>&& value) {
PushHelper<std::shared_ptr<T>>::f(vm, idx, std::move(value));
}
};
template <typename T>
struct PushHelper<std::shared_ptr<T>&> {
static inline void f(WrenVM* vm, int idx, std::shared_ptr<T>& value) {
PushHelper<std::shared_ptr<T>>::f(vm, idx, value);
}
};
template <typename T>
struct PushHelper<const std::shared_ptr<T>&> {
static inline void f(WrenVM* vm, int idx, const std::shared_ptr<T>& value) {
PushHelper<std::shared_ptr<T>>::f(vm, idx, value);
}
};
template <typename Iter>
inline void loopAndPushIterable(WrenVM* vm, const int idx, Iter begin, Iter end) {
using T = typename std::iterator_traits<Iter>::value_type;
wrenSetSlotNewList(vm, idx);
auto i = 0;
for (auto it = begin; it != end; ++it) {
PushHelper<T>::f(vm, idx + 1, *it);
wrenInsertInList(vm, idx, i++, idx + 1);
}
}
template <typename Iter>
inline void loopAndPushKeyPair(WrenVM* vm, const int idx, Iter begin, Iter end) {
using T = typename std::iterator_traits<Iter>::value_type;
using Key = typename T::first_type;
using Value = typename T::second_type;
wrenSetSlotNewMap(vm, idx);
wrenEnsureSlots(vm, 3);
for (auto it = begin; it != end; ++it) {
PushHelper<Key>::f(vm, idx + 1, std::forward<Key>(it->first));
PushHelper<Value>::f(vm, idx + 2, std::forward<Value>(it->second));
wrenSetMapValue(vm, idx, idx + 1, idx + 2);
}
}
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,372 @@
#pragma once
#include <algorithm>
#include <list>
#include <map>
#include <unordered_map>
#include <vector>
#include "module.hpp"
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename T, typename = void>
struct is_equality_comparable : std::false_type {};
template <typename T>
struct is_equality_comparable<
T, typename std::enable_if<true, decltype(std::declval<T&>() == std::declval<T&>(), (void)0)>::type>
: std::true_type {};
} // namespace detail
template <typename T, typename T2 = void>
class StdVectorHelper;
template <typename T>
class StdVectorHelper<T, typename std::enable_if<detail::is_equality_comparable<T>::value>::type> {
public:
static bool contains(std::vector<T>& self, const T& value) {
return std::find(self.begin(), self.end(), value) != self.end();
}
};
template <typename T>
class StdVectorHelper<T, typename std::enable_if<!detail::is_equality_comparable<T>::value>::type> {
public:
static bool contains(std::vector<T>& self, const T& value) {
return std::find_if(self.begin(), self.end(), [&](const T& e) -> bool { return &e == &value; }) != self.end();
}
};
#endif
template <typename T>
class StdVectorBindings {
public:
typedef typename std::vector<T>::iterator Iterator;
typedef typename std::vector<T> Vector;
static void setIndex(Vector& self, size_t index, T value) { self[index] = std::move(value); }
static const T& getIndex(Vector& self, size_t index) { return self[index]; }
static void add(Vector& self, T value) { self.push_back(std::move(value)); }
static std::variant<bool, Iterator> iterate(Vector& self, std::variant<std::nullptr_t, Iterator> other) {
if (other.index() == 1) {
auto it = std::get<Iterator>(other);
++it;
if (it != self.end()) {
return {it};
}
return {false};
} else {
if (self.empty()) return {false};
return {self.begin()};
}
}
static const T& iteratorValue(Vector& self, std::shared_ptr<Iterator> other) {
auto& it = *other;
return *it;
}
static size_t count(Vector& self) { return self.size(); }
static T removeAt(Vector& self, int32_t index) {
if (index == -1) {
auto ret = std::move(self.back());
self.pop_back();
return std::move(ret);
} else {
if (index < 0) {
index = static_cast<int32_t>(self.size()) + index;
}
if (index > static_cast<int32_t>(self.size())) {
throw std::out_of_range("invalid index");
} else if (index == static_cast<int32_t>(self.size())) {
auto ret = std::move(self.back());
self.pop_back();
return std::move(ret);
} else {
auto ret = std::move(self.at(index));
self.erase(self.begin() + index);
return std::move(ret);
}
}
}
static void insert(Vector& self, int32_t index, T value) {
if (index == -1) {
self.push_back(std::move(value));
} else {
if (index < 0) {
index = static_cast<int32_t>(self.size()) + index;
}
if (index > static_cast<int32_t>(self.size())) {
throw std::out_of_range("invalid index");
} else if (index == static_cast<int32_t>(self.size())) {
self.push_back(std::move(value));
} else {
auto it = self.begin() + index;
self.insert(it, std::move(value));
}
}
}
static bool contains(Vector& self, const T& value) {
// return std::find(self.begin(), self.end(), value) != self.end();
return StdVectorHelper<T>::contains(self, value);
}
static T pop(Vector& self) {
auto ret = std::move(self.back());
self.pop_back();
return std::move(ret);
}
static void clear(Vector& self) { self.clear(); }
static size_t size(Vector& self) { return self.size(); }
static bool empty(Vector& self) { return self.empty(); }
static void bind(ForeignModule& m, const std::string& name) {
auto& iter = m.klass<Iterator>(name + "Iter");
iter.ctor();
auto& cls = m.klass<Vector>(name);
cls.ctor();
cls.template funcExt<&StdVectorBindings<T>::getIndex>(OPERATOR_GET_INDEX);
cls.template funcExt<&StdVectorBindings<T>::setIndex>(OPERATOR_SET_INDEX);
cls.template funcExt<&StdVectorBindings<T>::add>("add");
cls.template funcExt<&StdVectorBindings<T>::iterate>("iterate");
cls.template funcExt<&StdVectorBindings<T>::iteratorValue>("iteratorValue");
cls.template funcExt<&StdVectorBindings<T>::removeAt>("removeAt");
cls.template funcExt<&StdVectorBindings<T>::insert>("insert");
cls.template funcExt<&StdVectorBindings<T>::contains>("contains");
cls.template funcExt<&StdVectorBindings<T>::pop>("pop");
cls.template funcExt<&StdVectorBindings<T>::clear>("clear");
cls.template funcExt<&StdVectorBindings<T>::size>("size");
cls.template funcExt<&StdVectorBindings<T>::empty>("empty");
cls.template propReadonlyExt<&StdVectorBindings<T>::count>("count");
}
};
template <typename T>
class StdListBindings {
public:
typedef typename std::list<T>::iterator Iterator;
typedef typename std::list<T> List;
static void setIndex(List& self, size_t index, T value) {
auto it = self.begin();
std::advance(it, index);
*it = std::move(value);
}
static const T& getIndex(List& self, size_t index) {
auto it = self.begin();
std::advance(it, index);
return *it;
}
static void add(List& self, T value) { self.push_back(std::move(value)); }
static std::variant<bool, Iterator> iterate(List& self, std::variant<std::nullptr_t, Iterator> other) {
if (other.index() == 1) {
auto it = std::get<Iterator>(other);
++it;
if (it != self.end()) {
return {it};
}
return {false};
} else {
if (self.empty()) return {false};
return {self.begin()};
}
}
static const T& iteratorValue(List& self, std::shared_ptr<Iterator> other) {
auto& it = *other;
return *it;
}
static size_t count(List& self) { return self.size(); }
static T removeAt(List& self, int32_t index) {
if (index == -1) {
auto ret = std::move(self.back());
self.pop_back();
return std::move(ret);
} else {
if (index < 0) {
index = static_cast<int32_t>(self.size()) + index;
}
if (index > static_cast<int32_t>(self.size())) {
throw std::out_of_range("invalid index");
} else if (index == static_cast<int32_t>(self.size())) {
auto ret = std::move(self.back());
self.pop_back();
return std::move(ret);
} else {
auto it = self.begin();
std::advance(it, index);
auto ret = std::move(*it);
self.erase(it);
return std::move(ret);
}
}
}
static void insert(List& self, int32_t index, T value) {
if (index == -1) {
self.push_back(std::move(value));
} else {
if (index < 0) {
index = static_cast<int32_t>(self.size()) + index;
}
if (index > static_cast<int32_t>(self.size())) {
throw std::out_of_range("invalid index");
} else if (index == static_cast<int32_t>(self.size())) {
self.push_back(std::move(value));
} else {
auto it = self.begin();
std::advance(it, index);
self.insert(it, std::move(value));
}
}
}
static bool contains(List& self, const T& value) {
return std::find(self.begin(), self.end(), value) != self.end();
}
static T pop(List& self) {
auto ret = std::move(self.back());
self.pop_back();
return std::move(ret);
}
static void clear(List& self) { self.clear(); }
static size_t size(List& self) { return self.size(); }
static bool empty(List& self) { return self.empty(); }
static void bind(ForeignModule& m, const std::string& name) {
auto& iter = m.klass<Iterator>(name + "Iter");
iter.ctor();
auto& cls = m.klass<List>(name);
cls.ctor();
cls.template funcExt<&StdListBindings<T>::getIndex>(OPERATOR_GET_INDEX);
cls.template funcExt<&StdListBindings<T>::setIndex>(OPERATOR_SET_INDEX);
cls.template funcExt<&StdListBindings<T>::add>("add");
cls.template funcExt<&StdListBindings<T>::iterate>("iterate");
cls.template funcExt<&StdListBindings<T>::iteratorValue>("iteratorValue");
cls.template funcExt<&StdListBindings<T>::removeAt>("removeAt");
cls.template funcExt<&StdListBindings<T>::insert>("insert");
cls.template funcExt<&StdListBindings<T>::contains>("contains");
cls.template funcExt<&StdListBindings<T>::pop>("pop");
cls.template funcExt<&StdListBindings<T>::clear>("clear");
cls.template funcExt<&StdListBindings<T>::size>("size");
cls.template funcExt<&StdListBindings<T>::empty>("empty");
cls.template propReadonlyExt<&StdListBindings<T>::count>("count");
}
};
template <typename Map>
class AbstractMapBindings {
public:
typedef typename Map::key_type K;
typedef typename Map::mapped_type T;
typedef typename Map::iterator Iterator;
typedef typename Map::value_type Pair;
static void setIndex(Map& self, const K& key, T value) { self[key] = std::move(value); }
static T& getIndex(Map& self, const K& key) { return self[key]; }
static std::variant<T, std::nullptr_t> remove(Map& self, const K& key) {
auto it = self.find(key);
if (it != self.end()) {
auto ret = std::move(it->second);
self.erase(it);
return {ret};
} else {
return {nullptr};
}
}
static bool containsKey(Map& self, const K& key) { return self.find(key) != self.end(); }
static size_t count(Map& self) { return self.size(); }
static void clear(Map& self) { self.clear(); }
static size_t size(Map& self) { return self.size(); }
static bool empty(Map& self) { return self.empty(); }
static std::variant<bool, Iterator> iterate(Map& self, std::variant<std::nullptr_t, Iterator> other) {
if (other.index() == 1) {
auto it = std::get<Iterator>(other);
++it;
if (it != self.end()) {
return {it};
}
return {false};
} else {
return {self.begin()};
}
}
static Pair iteratorValue(Map& self, std::shared_ptr<Iterator> other) {
auto& it = *other;
return *it;
}
static const K& pairKey(Pair& pair) { return pair.first; }
static const T& pairValue(Pair& pair) { return pair.second; }
static void bind(ForeignModule& m, const std::string& name) {
auto& pair = m.klass<Pair>(name + "Pair");
pair.template propReadonlyExt<&AbstractMapBindings<Map>::pairKey>("key");
pair.template propReadonlyExt<&AbstractMapBindings<Map>::pairValue>("value");
auto& iter = m.klass<Iterator>(name + "Iter");
iter.ctor();
auto& cls = m.klass<Map>(name);
cls.ctor();
cls.template funcExt<&AbstractMapBindings<Map>::getIndex>(OPERATOR_GET_INDEX);
cls.template funcExt<&AbstractMapBindings<Map>::setIndex>(OPERATOR_SET_INDEX);
cls.template funcExt<&AbstractMapBindings<Map>::remove>("remove");
cls.template funcExt<&AbstractMapBindings<Map>::containsKey>("containsKey");
cls.template funcExt<&AbstractMapBindings<Map>::iterate>("iterate");
cls.template funcExt<&AbstractMapBindings<Map>::iteratorValue>("iteratorValue");
cls.template funcExt<&AbstractMapBindings<Map>::clear>("clear");
cls.template funcExt<&AbstractMapBindings<Map>::size>("size");
cls.template funcExt<&AbstractMapBindings<Map>::empty>("empty");
cls.template propReadonlyExt<&AbstractMapBindings<Map>::count>("count");
}
};
template <typename K, typename V>
using StdMapBindings = AbstractMapBindings<std::map<K, V>>;
template <typename K, typename V>
using StdUnorderedMapBindings = AbstractMapBindings<std::unordered_map<K, V>>;
} // namespace wrenbind17

View File

@ -0,0 +1,71 @@
#pragma once
#include <deque>
#include <wren.hpp>
#include "pop.hpp"
#include "push.hpp"
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename T>
struct PushHelper<std::deque<T>> {
static inline void f(WrenVM* vm, int idx, std::deque<T> value) {
if (isClassRegistered(vm, typeid(std::deque<T>).hash_code())) {
pushAsMove<std::deque<T>>(vm, idx, std::move(value));
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PushHelper<std::deque<T>*> {
static inline void f(WrenVM* vm, int idx, std::deque<T>* value) {
if (isClassRegistered(vm, typeid(std::deque<T>).hash_code())) {
pushAsPtr<std::deque<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value->begin(), value->end());
}
}
};
template <typename T>
struct PushHelper<const std::deque<T>&> {
static inline void f(WrenVM* vm, int idx, const std::deque<T>& value) {
if (isClassRegistered(vm, typeid(std::deque<T>).hash_code())) {
pushAsConstRef<std::deque<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PopHelper<const std::deque<T>&> {
static inline std::deque<T> f(WrenVM* vm, const int idx) {
const auto type = wrenGetSlotType(vm, idx);
if (type == WrenType::WREN_TYPE_FOREIGN) {
return *getSlotForeign<std::deque<T>>(vm, idx).get();
}
if (type != WrenType::WREN_TYPE_LIST) throw BadCast("Bad cast when getting value from Wren expected list");
std::deque<T> res;
const auto size = wrenGetListCount(vm, idx);
wrenEnsureSlots(vm, 1);
for (size_t i = 0; i < size; i++) {
wrenGetListElement(vm, idx, static_cast<int>(i), idx + 1);
res.push_back(PopHelper<T>::f(vm, idx + 1));
}
return res;
}
};
template <typename T>
struct PopHelper<std::deque<T>> {
static inline std::deque<T> f(WrenVM* vm, const int idx) { return PopHelper<std::deque<T>>::f(vm, idx); }
};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,71 @@
#pragma once
#include <list>
#include <wren.hpp>
#include "pop.hpp"
#include "push.hpp"
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename T>
struct PushHelper<std::list<T>> {
static inline void f(WrenVM* vm, int idx, std::list<T> value) {
if (isClassRegistered(vm, typeid(std::list<T>).hash_code())) {
pushAsMove<std::list<T>>(vm, idx, std::move(value));
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PushHelper<std::list<T>*> {
static inline void f(WrenVM* vm, int idx, std::list<T>* value) {
if (isClassRegistered(vm, typeid(std::list<T>).hash_code())) {
pushAsPtr<std::list<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value->begin(), value->end());
}
}
};
template <typename T>
struct PushHelper<const std::list<T>&> {
static inline void f(WrenVM* vm, int idx, const std::list<T>& value) {
if (isClassRegistered(vm, typeid(std::list<T>).hash_code())) {
pushAsConstRef<std::list<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PopHelper<const std::list<T>&> {
static inline std::list<T> f(WrenVM* vm, const int idx) {
const auto type = wrenGetSlotType(vm, idx);
if (type == WrenType::WREN_TYPE_FOREIGN) {
return *getSlotForeign<std::list<T>>(vm, idx).get();
}
if (type != WrenType::WREN_TYPE_LIST) throw BadCast("Bad cast when getting value from Wren expected list");
std::list<T> res;
const auto size = wrenGetListCount(vm, idx);
wrenEnsureSlots(vm, 1);
for (size_t i = 0; i < size; i++) {
wrenGetListElement(vm, idx, static_cast<int>(i), idx + 1);
res.push_back(PopHelper<T>::f(vm, idx + 1));
}
return res;
}
};
template <typename T>
struct PopHelper<std::list<T>> {
static inline std::list<T> f(WrenVM* vm, const int idx) { return PopHelper<const std::list<T>&>::f(vm, idx); }
};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,80 @@
#pragma once
#include <map>
#include <unordered_map>
#include <wren.hpp>
#include "pop.hpp"
#include "push.hpp"
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename T>
struct PushHelper<std::map<std::string, T>> {
static inline void f(WrenVM* vm, int idx, std::map<std::string, T> value) {
if (isClassRegistered(vm, typeid(std::map<std::string, T>).hash_code())) {
pushAsMove<std::map<std::string, T>>(vm, idx, std::move(value));
} else {
loopAndPushKeyPair(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PushHelper<std::map<std::string, T>*> {
static inline void f(WrenVM* vm, int idx, std::map<std::string, T>* value) {
if (isClassRegistered(vm, typeid(std::map<std::string, T>).hash_code())) {
pushAsPtr<std::map<std::string, T>>(vm, idx, value);
} else {
loopAndPushKeyPair(vm, idx, value->begin(), value->end());
}
}
};
template <typename T>
struct PushHelper<const std::map<std::string, T>&> {
static inline void f(WrenVM* vm, int idx, const std::map<std::string, T>& value) {
if (isClassRegistered(vm, typeid(std::map<std::string, T>).hash_code())) {
pushAsConstRef<std::map<std::string, T>>(vm, idx, value);
} else {
loopAndPushKeyPair(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PushHelper<std::unordered_map<std::string, T>> {
static inline void f(WrenVM* vm, int idx, std::unordered_map<std::string, T> value) {
if (isClassRegistered(vm, typeid(std::unordered_map<std::string, T>).hash_code())) {
pushAsMove<std::unordered_map<std::string, T>>(vm, idx, std::move(value));
} else {
loopAndPushKeyPair(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PushHelper<std::unordered_map<std::string, T>*> {
static inline void f(WrenVM* vm, int idx, std::unordered_map<std::string, T>* value) {
if (isClassRegistered(vm, typeid(std::unordered_map<std::string, T>).hash_code())) {
pushAsPtr<std::unordered_map<std::string, T>>(vm, idx, value);
} else {
loopAndPushKeyPair(vm, idx, value->begin(), value->end());
}
}
};
template <typename T>
struct PushHelper<const std::unordered_map<std::string, T>&> {
static inline void f(WrenVM* vm, int idx, const std::unordered_map<std::string, T>& value) {
if (isClassRegistered(vm, typeid(std::unordered_map<std::string, T>).hash_code())) {
pushAsConstRef<std::unordered_map<std::string, T>>(vm, idx, value);
} else {
loopAndPushKeyPair(vm, idx, value.begin(), value.end());
}
}
};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,61 @@
#pragma once
#include <optional>
#include <wren.hpp>
#include "pop.hpp"
#include "push.hpp"
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename T>
struct PushHelper<std::optional<T>> {
inline static void f(WrenVM* vm, int idx, const std::optional<T>& value) {
if (value.has_value()) {
PushHelper<T>::f(vm, idx, value.value());
} else {
PushHelper<std::nullptr_t>::f(vm, idx, nullptr);
}
}
};
template <typename T>
struct PushHelper<std::optional<T>&> {
inline static void f(WrenVM* vm, int idx, const std::optional<T>& value) {
PushHelper<std::optional<T>>::f(vm, idx, value);
}
};
template <typename T>
struct PushHelper<std::optional<T>*> {
inline static void f(WrenVM* vm, int idx, const std::optional<T>* value) {
PushHelper<std::optional<T>>::f(vm, idx, *value);
}
};
template <typename T>
struct PushHelper<const std::optional<T>&> {
inline static void f(WrenVM* vm, int idx, const std::optional<T>& value) {
PushHelper<std::optional<T>>::f(vm, idx, value);
}
};
template <typename T>
struct PopHelper<std::optional<T>> {
static inline std::optional<T> f(WrenVM* vm, const int idx) {
if (is<std::nullptr_t>(vm, idx)) {
return std::nullopt;
} else {
return PopHelper<T>::f(vm, idx);
}
}
};
template <typename T>
struct PopHelper<const std::optional<T>&> {
static inline std::optional<T> f(WrenVM* vm, const int idx) { return PopHelper<std::optional<T>>::f(vm, idx); }
};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,134 @@
#pragma once
#include <set>
#include <unordered_set>
#include <wren.hpp>
#include "pop.hpp"
#include "push.hpp"
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename T>
struct PushHelper<std::set<T>> {
static inline void f(WrenVM* vm, int idx, std::set<T> value) {
if (isClassRegistered(vm, typeid(std::set<T>).hash_code())) {
pushAsMove<std::vector<T>>(vm, idx, std::move(value));
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PushHelper<std::set<T>*> {
static inline void f(WrenVM* vm, int idx, std::set<T>* value) {
if (isClassRegistered(vm, typeid(std::set<T>).hash_code())) {
pushAsPtr<std::set<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value->begin(), value->end());
}
}
};
template <typename T>
struct PushHelper<const std::set<T>&> {
static inline void f(WrenVM* vm, int idx, const std::set<T>& value) {
if (isClassRegistered(vm, typeid(std::set<T>).hash_code())) {
pushAsConstRef<std::set<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PopHelper<const std::set<T>&> {
static inline std::set<T> f(WrenVM* vm, const int idx) {
const auto type = wrenGetSlotType(vm, idx);
if (type == WrenType::WREN_TYPE_FOREIGN) {
return *getSlotForeign<std::set<T>>(vm, idx).get();
}
if (type != WrenType::WREN_TYPE_LIST) throw BadCast("Bad cast when getting value from Wren expected list");
std::set<T> res;
const auto size = wrenGetListCount(vm, idx);
wrenEnsureSlots(vm, 1);
res.reserve(size);
for (size_t i = 0; i < size; i++) {
wrenGetListElement(vm, idx, static_cast<int>(i), idx + 1);
res.insert(PopHelper<T>::f(vm, idx + 1));
}
return res;
}
};
template <typename T>
struct PopHelper<std::set<T>> {
static inline std::set<T> f(WrenVM* vm, const int idx) { return PopHelper<const std::set<T>&>::f(vm, idx); }
};
template <typename T>
struct PushHelper<std::unordered_set<T>> {
static inline void f(WrenVM* vm, int idx, std::unordered_set<T> value) {
if (isClassRegistered(vm, typeid(std::unordered_set<T>).hash_code())) {
pushAsMove<std::vector<T>>(vm, idx, std::move(value));
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PushHelper<std::unordered_set<T>*> {
static inline void f(WrenVM* vm, int idx, std::unordered_set<T>* value) {
if (isClassRegistered(vm, typeid(std::unordered_set<T>).hash_code())) {
pushAsPtr<std::set<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value->begin(), value->end());
}
}
};
template <typename T>
struct PushHelper<const std::unordered_set<T>&> {
static inline void f(WrenVM* vm, int idx, const std::unordered_set<T>& value) {
if (isClassRegistered(vm, typeid(std::unordered_set<T>).hash_code())) {
pushAsConstRef<std::set<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PopHelper<const std::unordered_set<T>&> {
static inline std::set<T> f(WrenVM* vm, const int idx) {
const auto type = wrenGetSlotType(vm, idx);
if (type == WrenType::WREN_TYPE_FOREIGN) {
return *getSlotForeign<std::unordered_set<T>>(vm, idx).get();
}
if (type != WrenType::WREN_TYPE_LIST) throw BadCast("Bad cast when getting value from Wren expected list");
std::unordered_set<T> res;
const auto size = wrenGetListCount(vm, idx);
wrenEnsureSlots(vm, 1);
res.reserve(size);
for (size_t i = 0; i < size; i++) {
wrenGetListElement(vm, idx, static_cast<int>(i), idx + 1);
res.insert(PopHelper<T>::f(vm, idx + 1));
}
return res;
}
};
template <typename T>
struct PopHelper<std::unordered_set<T>> {
static inline std::set<T> f(WrenVM* vm, const int idx) {
return PopHelper<const std::unordered_set<T>&>::f(vm, idx);
}
};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,84 @@
#pragma once
#include <variant>
#include <wren.hpp>
#include "pop.hpp"
#include "push.hpp"
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename VariantType>
inline void loopAndPushVariant(WrenVM* vm, int idx, const VariantType& v, size_t i) {
PushHelper<std::nullptr_t>::f(vm, idx, nullptr);
}
template <typename VariantType, typename T, typename... Ts>
inline void loopAndPushVariant(WrenVM* vm, int idx, const VariantType& v, size_t i) {
if (v.index() == i) {
PushHelper<T>::f(vm, idx, std::get<T>(v));
} else {
loopAndPushVariant<VariantType, Ts...>(vm, idx, v, i + 1);
}
}
template <typename... Ts>
struct PushHelper<std::variant<Ts...>> {
inline static void f(WrenVM* vm, int idx, const std::variant<Ts...>& value) {
loopAndPushVariant<std::variant<Ts...>, Ts...>(vm, idx, value, 0);
}
};
template <typename... Ts>
struct PushHelper<std::variant<Ts...>&> {
inline static void f(WrenVM* vm, int idx, const std::variant<Ts...>& value) {
PushHelper<std::variant<Ts...>>::f(vm, idx, value);
}
};
template <typename... Ts>
struct PushHelper<std::variant<Ts...>*> {
inline static void f(WrenVM* vm, int idx, const std::variant<Ts...>* value) {
PushHelper<std::variant<Ts...>>::f(vm, idx, *value);
}
};
template <typename... Ts>
struct PushHelper<const std::variant<Ts...>&> {
inline static void f(WrenVM* vm, int idx, const std::variant<Ts...>& value) {
PushHelper<std::variant<Ts...>>::f(vm, idx, value);
}
};
template <typename VariantType>
VariantType loopAndFindVariant(WrenVM* vm, int idx) {
throw BadCast("Bad cast when getting variant from Wren");
}
template <typename VariantType, typename T, typename... Ts>
VariantType loopAndFindVariant(WrenVM* vm, const int idx) {
if (CheckSlot<T>::f(vm, idx)) {
return {PopHelper<T>::f(vm, idx)};
}
return loopAndFindVariant<VariantType, Ts...>(vm, idx);
}
template <typename... Ts>
struct PopHelper<std::variant<Ts...>> {
static inline std::variant<Ts...> f(WrenVM* vm, const int idx) {
using VariantType = typename std::variant<Ts...>;
return loopAndFindVariant<VariantType, Ts...>(vm, idx);
}
};
template <typename... Ts>
struct PopHelper<const std::variant<Ts...>&> {
static inline std::variant<Ts...> f(WrenVM* vm, const int idx) {
using VariantType = typename std::variant<Ts...>;
return loopAndFindVariant<VariantType, Ts...>(vm, idx);
}
};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,72 @@
#pragma once
#include <vector>
#include <wren.hpp>
#include "pop.hpp"
#include "push.hpp"
namespace wrenbind17 {
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace detail {
template <typename T>
struct PushHelper<std::vector<T>> {
static inline void f(WrenVM* vm, int idx, std::vector<T> value) {
if (isClassRegistered(vm, typeid(std::vector<T>).hash_code())) {
pushAsMove<std::vector<T>>(vm, idx, std::move(value));
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PushHelper<std::vector<T>*> {
static inline void f(WrenVM* vm, int idx, std::vector<T>* value) {
if (isClassRegistered(vm, typeid(std::vector<T>).hash_code())) {
pushAsPtr<std::vector<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value->begin(), value->end());
}
}
};
template <typename T>
struct PushHelper<const std::vector<T>&> {
static inline void f(WrenVM* vm, int idx, const std::vector<T>& value) {
if (isClassRegistered(vm, typeid(std::vector<T>).hash_code())) {
pushAsConstRef<std::vector<T>>(vm, idx, value);
} else {
loopAndPushIterable(vm, idx, value.begin(), value.end());
}
}
};
template <typename T>
struct PopHelper<const std::vector<T>&> {
static inline std::vector<T> f(WrenVM* vm, const int idx) {
const auto type = wrenGetSlotType(vm, idx);
if (type == WrenType::WREN_TYPE_FOREIGN) {
return *getSlotForeign<std::vector<T>>(vm, idx).get();
}
if (type != WrenType::WREN_TYPE_LIST) throw BadCast("Bad cast when getting value from Wren expected list");
std::vector<T> res;
const auto size = wrenGetListCount(vm, idx);
wrenEnsureSlots(vm, 1);
res.reserve(size);
for (size_t i = 0; i < size; i++) {
wrenGetListElement(vm, idx, static_cast<int>(i), idx + 1);
res.push_back(PopHelper<T>::f(vm, idx + 1));
}
return res;
}
};
template <typename T>
struct PopHelper<std::vector<T>> {
static inline std::vector<T> f(WrenVM* vm, const int idx) { return PopHelper<const std::vector<T>&>::f(vm, idx); }
};
} // namespace detail
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,97 @@
#pragma once
#include <memory>
#include "exception.hpp"
#include "method.hpp"
#include "pop.hpp"
#include "push.hpp"
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
/**
* @ingroup wrenbind17
* @brief Holds some Wren variable which can be a class or class instance
* @details You can use this to pass around Wren classes or class instances.
* You can also use this to get Wren class methods. You can also call a Wren
* function from C++ side and pass this Variable into Wren. To get this variable,
* either call a Wren function that returns some class (or class instance), or
* use wrenbind17::VM::find() function that looks up a class (or class instance)
* based on the module name.
* @note This variable can safely outlive the wrenbind17::VM class. If that happens
* then functions of this class will throw wrenbind17::RuntimeError exception. This
* holder will not try to free the Wren variable if the VM has been terminated. You
* don't have to worry about the lifetime of this holder. (uses weak pointers).
*/
class Variable {
public:
Variable() {}
Variable(const std::shared_ptr<Handle>& handle) : handle(handle) {}
~Variable() { reset(); }
/*!
* @brief Looks up a function from this Wren variable.
* @details The signature must match Wren function signature.
* For example: `main()` or `foo(_,_)` etc. Use underscores to specify
* parameters of that function.
* @throws RuntimeError if this variable is invalid or the Wren VM has terminated.
*/
Method func(const std::string& signature) {
if (const auto ptr = handle->getVmWeak().lock()) {
auto* h = wrenMakeCallHandle(ptr.get(), signature.c_str());
return Method(handle, std::make_shared<Handle>(ptr, h));
} else {
throw RuntimeError("Invalid handle");
}
}
Handle& getHandle() { return *handle; }
const Handle& getHandle() const { return *handle; }
operator bool() const { return handle.operator bool(); }
void reset() { handle.reset(); }
private:
std::shared_ptr<Handle> handle;
};
template <>
inline Variable detail::getSlot<Variable>(WrenVM* vm, const int idx) {
validate<WrenType::WREN_TYPE_UNKNOWN>(vm, idx);
return Variable(std::make_shared<Handle>(getSharedVm(vm), wrenGetSlotHandle(vm, idx)));
}
template <>
inline bool detail::is<Variable>(WrenVM* vm, const int idx) {
return wrenGetSlotType(vm, idx) == WREN_TYPE_UNKNOWN;
}
template <>
inline void detail::PushHelper<Variable>::f(WrenVM* vm, int idx, const Variable& value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
template <>
inline void detail::PushHelper<Variable>::f(WrenVM* vm, int idx, Variable&& value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
template <>
inline void detail::PushHelper<const Variable>::f(WrenVM* vm, int idx, const Variable value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
template <>
inline void detail::PushHelper<const Variable&>::f(WrenVM* vm, int idx, const Variable& value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
template <>
inline void detail::PushHelper<Variable&>::f(WrenVM* vm, int idx, Variable& value) {
wrenSetSlotHandle(value.getHandle().getVm(), idx, value.getHandle().getHandle());
}
} // namespace wrenbind17

View File

@ -0,0 +1,422 @@
#pragma once
#include <cassert>
#include <cstdlib>
#include <cstring>
#include <fstream>
#include <functional>
#include <iostream>
#include <memory>
#include <sstream>
#include <unordered_map>
#include <vector>
#include <wren.hpp>
#include "exception.hpp"
#include "map.hpp"
#include "module.hpp"
#include "variable.hpp"
#ifndef DOXYGEN_SHOULD_SKIP_THIS
namespace std {
template <>
struct hash<std::pair<size_t, size_t>> {
inline size_t operator()(const std::pair<size_t, size_t>& v) const {
const std::hash<size_t> hasher;
return hasher(v.first) ^ hasher(v.second);
}
};
} // namespace std
#endif
/**
* @ingroup wrenbind17
*/
namespace wrenbind17 {
/**
* @ingroup wrenbind17
*/
typedef std::function<void(const char*)> PrintFn;
/**
* @ingroup wrenbind17
*/
typedef std::function<std::string(const std::vector<std::string>& paths, const std::string& name)> LoadFileFn;
/**
* @ingroup wrenbind17
* @brief Holds the entire Wren VM from which all of the magic happens
*/
class VM {
public:
/*!
* @brief The only constructor available
* @param paths The lookup paths used by the import loader function
* @param initHeap The size of the heap at the beginning
* @param minHeap The minimum size of the heap
* @param heapGrowth How the heap should grow
*/
inline explicit VM(std::vector<std::string> paths = {"./"}, const size_t initHeap = 1024 * 1024,
const size_t minHeap = 1024 * 1024 * 10, const int heapGrowth = 50)
: data(std::make_unique<Data>()) {
data->paths = std::move(paths);
data->printFn = [](const char* text) -> void { std::cout << text; };
data->loadFileFn = [](const std::vector<std::string>& paths, const std::string& name) -> std::string {
for (const auto& path : paths) {
const auto test = path + "/" + std::string(name) + ".wren";
std::ifstream t(test);
if (!t) continue;
std::string source((std::istreambuf_iterator<char>(t)), std::istreambuf_iterator<char>());
return source;
}
throw NotFound();
};
wrenInitConfiguration(&data->config);
data->config.initialHeapSize = initHeap;
data->config.minHeapSize = minHeap;
data->config.heapGrowthPercent = heapGrowth;
data->config.userData = data.get();
#if WREN_VERSION_NUMBER >= 4000 // >= 0.4.0
data->config.reallocateFn = [](void* memory, size_t newSize, void* userData) -> void* {
return std::realloc(memory, newSize);
};
data->config.loadModuleFn = [](WrenVM* vm, const char* name) -> WrenLoadModuleResult {
auto res = WrenLoadModuleResult();
auto& self = *reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
const auto mod = self.modules.find(name);
if (mod != self.modules.end()) {
auto source = mod->second.str();
auto buffer = new char[source.size() + 1];
std::memcpy(buffer, &source[0], source.size() + 1);
res.source = buffer;
res.onComplete = [](WrenVM* vm, const char* name, struct WrenLoadModuleResult result) {
delete[] result.source;
};
return res;
}
try {
auto source = self.loadFileFn(self.paths, std::string(name));
auto buffer = new char[source.size() + 1];
std::memcpy(buffer, &source[0], source.size() + 1);
res.source = buffer;
res.onComplete = [](WrenVM* vm, const char* name, struct WrenLoadModuleResult result) {
delete[] result.source;
};
} catch (std::exception& e) {
(void)e;
}
return res;
};
#else // < 0.4.0
data->config.reallocateFn = std::realloc;
data->config.loadModuleFn = [](WrenVM* vm, const char* name) -> char* {
auto& self = *reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
const auto mod = self.modules.find(name);
if (mod != self.modules.end()) {
auto source = mod->second.str();
auto buffer = new char[source.size() + 1];
std::memcpy(buffer, &source[0], source.size() + 1);
return buffer;
}
try {
auto source = self.loadFileFn(self.paths, std::string(name));
auto buffer = new char[source.size() + 1];
std::memcpy(buffer, &source[0], source.size() + 1);
return buffer;
} catch (std::exception& e) {
(void)e;
return nullptr;
}
};
#endif // WREN_VERSION_NUMBER >= 4000
data->config.bindForeignMethodFn = [](WrenVM* vm, const char* module, const char* className,
const bool isStatic, const char* signature) -> WrenForeignMethodFn {
auto& self = *reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
try {
auto& found = self.modules.at(module);
auto& klass = found.findKlass(className);
return klass.findSignature(signature, isStatic);
} catch (...) {
std::cerr << "Wren foreign method " << signature << " not found in C++" << std::endl;
std::abort();
return nullptr;
}
};
data->config.bindForeignClassFn = [](WrenVM* vm, const char* module,
const char* className) -> WrenForeignClassMethods {
auto& self = *reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
try {
auto& found = self.modules.at(module);
auto& klass = found.findKlass(className);
return klass.getAllocators();
} catch (...) {
exceptionHandler(vm, std::current_exception());
return WrenForeignClassMethods{nullptr, nullptr};
}
};
data->config.writeFn = [](WrenVM* vm, const char* text) {
auto& self = *reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
self.printFn(text);
};
data->config.errorFn = [](WrenVM* vm, WrenErrorType type, const char* module, const int line,
const char* message) {
auto& self = *reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
std::stringstream ss;
switch (type) {
case WREN_ERROR_COMPILE:
ss << "Compile error: " << message << " at " << module << ":" << line << "\n";
break;
case WREN_ERROR_RUNTIME:
if (!self.nextError.empty()) {
ss << "Runtime error: " << self.nextError << "\n";
self.nextError.clear();
} else {
ss << "Runtime error: " << message << "\n";
}
break;
case WREN_ERROR_STACK_TRACE:
ss << " at: " << module << ":" << line << "\n";
break;
default:
break;
}
self.lastError += ss.str();
};
data->vm = std::shared_ptr<WrenVM>(wrenNewVM(&data->config), [](WrenVM* ptr) { wrenFreeVM(ptr); });
}
inline VM(const VM& other) = delete;
inline VM(VM&& other) noexcept { swap(other); }
inline ~VM() = default;
inline VM& operator=(const VM& other) = delete;
inline VM& operator=(VM&& other) noexcept {
if (this != &other) {
swap(other);
}
return *this;
}
inline void swap(VM& other) noexcept { std::swap(data, other.data); }
/*!
* @brief Runs a Wren source code by passing it as a string
* @param name The module name to assign this code into, this module name
* can be then used to import this code in some other place
* @param code Your raw multiline Wren code
* @throws CompileError if the compilation has failed
*/
inline void runFromSource(const std::string& name, const std::string& code) {
const auto result = wrenInterpret(data->vm.get(), name.c_str(), code.c_str());
if (result != WREN_RESULT_SUCCESS) {
throw CompileError(getLastError());
}
return;
}
/*!
* @brief Runs a Wren source code directly from a file
* @param name The module name to assign this code into, this module name
* can be then used to import this code in some other place
* @param path The path to the file
* @throws Exception if the file has not been found or the file cannot be read
* @throws CompileError if the compilation has failed
*/
inline void runFromFile(const std::string& name, const std::string& path) {
std::ifstream t(path);
if (!t) throw Exception("Compile error: Failed to open source file");
std::string str((std::istreambuf_iterator<char>(t)), std::istreambuf_iterator<char>());
runFromSource(name, str);
}
/*!
* @brief Runs a Wren source code by passing it as a string
* @see setLoadFileFunc
* @param name The module name to load, this will use the loader function to
* load the file from
* @throws CompileError if the compilation has failed
*/
inline void runFromModule(const std::string& name) {
const auto source = data->loadFileFn(data->paths, name);
runFromSource(name, source);
}
/*!
* @brief Looks up a variable from a module
* @param module The name of the module to look for the variable in
* @param name The name of the variable or a class itself that must start
* with a capital letter
* @throws NotFound if the variable has not been found
*/
inline Variable find(const std::string& module, const std::string& name) {
wrenEnsureSlots(data->vm.get(), 1);
wrenGetVariable(data->vm.get(), module.c_str(), name.c_str(), 0);
auto* handle = wrenGetSlotHandle(data->vm.get(), 0);
if (!handle) throw NotFound();
return Variable(std::make_shared<Handle>(data->vm, handle));
}
/*!
* @brief Creates a new custom module
* @note Calling this function multiple times with the same name
* does not create a new module, but instead it returns the same module.
*/
inline ForeignModule& module(const std::string& name) {
auto it = data->modules.find(name);
if (it == data->modules.end()) {
it = data->modules.insert(std::make_pair(name, ForeignModule(name, data->vm.get()))).first;
}
return it->second;
}
inline void addClassType(const std::string& module, const std::string& name, const size_t hash) {
data->addClassType(module, name, hash);
}
inline void getClassType(std::string& module, std::string& name, const size_t hash) {
data->getClassType(module, name, hash);
}
inline bool isClassRegistered(const size_t hash) const { return data->isClassRegistered(hash); }
inline void addClassCast(std::shared_ptr<detail::ForeignPtrConvertor> convertor, const size_t hash,
const size_t other) {
data->addClassCast(std::move(convertor), hash, other);
}
inline detail::ForeignPtrConvertor* getClassCast(const size_t hash, const size_t other) {
return data->getClassCast(hash, other);
}
inline std::string getLastError() { return data->getLastError(); }
inline void setNextError(std::string str) { data->setNextError(std::move(str)); }
/*!
* @brief Set a custom print function that is used by the System.print()
* @see PrintFn
*/
inline void setPrintFunc(const PrintFn& fn) { data->printFn = fn; }
/*!
* @brief Set a custom loader function for imports
* @see LoadFileFn
* @details This must be a function that accepts a std::vector of strings
* (which are the lookup paths from the constructor) and the name of the import
* as the second parameter. You must return the source code from this custom function.
* If you want to cancel the import, simply throw an exception.
*/
inline void setLoadFileFunc(const LoadFileFn& fn) { data->loadFileFn = fn; }
/*!
* @brief Runs the garbage collector
*/
inline void gc() { wrenCollectGarbage(data->vm.get()); }
class Data {
public:
std::shared_ptr<WrenVM> vm;
WrenConfiguration config;
std::vector<std::string> paths;
std::unordered_map<std::string, ForeignModule> modules;
std::unordered_map<size_t, std::string> classToModule;
std::unordered_map<size_t, std::string> classToName;
std::unordered_map<std::pair<size_t, size_t>, std::shared_ptr<detail::ForeignPtrConvertor>> classCasting;
std::string lastError;
std::string nextError;
PrintFn printFn;
LoadFileFn loadFileFn;
inline void addClassType(const std::string& module, const std::string& name, const size_t hash) {
classToModule.insert(std::make_pair(hash, module));
classToName.insert(std::make_pair(hash, name));
}
inline void getClassType(std::string& module, std::string& name, const size_t hash) {
module = classToModule.at(hash);
name = classToName.at(hash);
}
inline bool isClassRegistered(const size_t hash) const {
return classToModule.find(hash) != classToModule.end();
}
inline void addClassCast(std::shared_ptr<detail::ForeignPtrConvertor> convertor, const size_t hash,
const size_t other) {
classCasting.insert(std::make_pair(std::make_pair(hash, other), std::move(convertor)));
}
inline detail::ForeignPtrConvertor* getClassCast(const size_t hash, const size_t other) {
return classCasting.at(std::pair(hash, other)).get();
}
inline std::string getLastError() {
std::string str;
std::swap(str, lastError);
return str;
}
inline void setNextError(std::string str) { nextError = std::move(str); }
};
private:
std::unique_ptr<Data> data;
};
#ifndef DOXYGEN_SHOULD_SKIP_THIS
inline std::shared_ptr<WrenVM> getSharedVm(WrenVM* vm) {
assert(vm);
auto self = reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
assert(self->vm);
return self->vm;
}
inline void addClassType(WrenVM* vm, const std::string& module, const std::string& name, const size_t hash) {
assert(vm);
auto self = reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
self->addClassType(module, name, hash);
}
inline void getClassType(WrenVM* vm, std::string& module, std::string& name, const size_t hash) {
assert(vm);
auto self = reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
self->getClassType(module, name, hash);
}
inline bool isClassRegistered(WrenVM* vm, const size_t hash) {
assert(vm);
auto self = reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
return self->isClassRegistered(hash);
}
inline void addClassCast(WrenVM* vm, std::shared_ptr<detail::ForeignPtrConvertor> convertor, const size_t hash,
const size_t other) {
assert(vm);
auto self = reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
self->addClassCast(std::move(convertor), hash, other);
}
inline detail::ForeignPtrConvertor* getClassCast(WrenVM* vm, const size_t hash, const size_t other) {
assert(vm);
auto self = reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
return self->getClassCast(hash, other);
}
inline std::string getLastError(WrenVM* vm) {
assert(vm);
auto self = reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
return self->getLastError();
}
inline void setNextError(WrenVM* vm, std::string str) {
assert(vm);
auto self = reinterpret_cast<VM::Data*>(wrenGetUserData(vm));
self->setNextError(std::move(str));
}
#endif
} // namespace wrenbind17

View File

@ -0,0 +1,16 @@
#pragma once
/**
* @defgroup wrenbind17
* @brief Wren lang binding library for C++17
*/
#include "std.hpp"
#include "stddeque.hpp"
#include "stdlist.hpp"
#include "stdmap.hpp"
#include "stdoptional.hpp"
#include "stdset.hpp"
#include "stdvariant.hpp"
#include "stdvector.hpp"
#include "vm.hpp"

View File

@ -21,7 +21,8 @@
namespace starrocks {
TEST(CommandExecutorTest, not_support) {
EXPECT_TRUE(execute_command("set_config_11", "{}").is_not_supported());
std::string result;
EXPECT_TRUE(execute_command("set_config_11", "{}", &result).is_not_supported());
}
} // namespace starrocks

View File

@ -0,0 +1,62 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// 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
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package com.starrocks.qe;
import com.starrocks.common.UserException;
import com.starrocks.proto.ExecuteCommandRequestPB;
import com.starrocks.proto.ExecuteCommandResultPB;
import com.starrocks.rpc.DataNodeServiceClient;
import com.starrocks.server.GlobalStateMgr;
import com.starrocks.sql.ast.ExecuteScriptStmt;
import com.starrocks.system.DataNode;
import com.starrocks.thrift.TNetworkAddress;
import org.apache.commons.lang.StringUtils;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
public class ExecuteScriptExecutor {
private static final Logger LOG = LogManager.getLogger(ExecuteScriptExecutor.class);
public static void execute(ExecuteScriptStmt stmt, ConnectContext ctx) throws UserException {
DataNode be = GlobalStateMgr.getCurrentSystemInfo().getBackend(stmt.getBeId());
TNetworkAddress address = new TNetworkAddress(be.getHost(), be.getBrpcPort());
ExecuteCommandRequestPB request = new ExecuteCommandRequestPB();
request.command = "execute_script";
request.params = stmt.getScript();
try {
Future<ExecuteCommandResultPB> future = DataNodeServiceClient.getInstance().executeCommand(address, request);
ExecuteCommandResultPB result = future.get(stmt.getTimeoutSec(), TimeUnit.SECONDS);
if (result.status.statusCode != 0) {
LOG.warn("execute script error BE: {} script:{} result: {}", stmt.getBeId(),
StringUtils.abbreviate(stmt.getScript(), 1000),
result.status.errorMsgs);
throw new UserException(result.status.toString());
} else {
LOG.info("execute script ok BE: {} script:{} result: {}", stmt.getBeId(),
StringUtils.abbreviate(stmt.getScript(), 1000), StringUtils.abbreviate(result.result, 1000));
ctx.getState().setOk(0, 0, result.result);
}
} catch (InterruptedException ie) {
LOG.warn("got interrupted exception when sending proxy request to " + address);
Thread.currentThread().interrupt();
throw new UserException("got interrupted exception when sending proxy request to " + address);
} catch (Exception e) {
throw new UserException("executeCommand RPC failed BE:" + address + " err " + e.getMessage());
}
}
}

View File

@ -112,6 +112,7 @@ import com.starrocks.sql.ast.DmlStmt;
import com.starrocks.sql.ast.DropHistogramStmt;
import com.starrocks.sql.ast.DropStatsStmt;
import com.starrocks.sql.ast.ExecuteAsStmt;
import com.starrocks.sql.ast.ExecuteScriptStmt;
import com.starrocks.sql.ast.ExportStmt;
import com.starrocks.sql.ast.InsertStmt;
import com.starrocks.sql.ast.KillAnalyzeStmt;
@ -541,6 +542,8 @@ public class StmtExecutor {
handleDelSqlBlackListStmt();
} else if (parsedStmt instanceof ExecuteAsStmt) {
handleExecAsStmt();
} else if (parsedStmt instanceof ExecuteScriptStmt) {
handleExecScriptStmt();
} else if (parsedStmt instanceof SetRoleStmt) {
handleSetRole();
} else if (parsedStmt instanceof SetDefaultRoleStmt) {
@ -988,6 +991,10 @@ public class StmtExecutor {
ExecuteAsExecutor.execute((ExecuteAsStmt) parsedStmt, context);
}
private void handleExecScriptStmt() throws PrivilegeException, UserException {
ExecuteScriptExecutor.execute((ExecuteScriptStmt) parsedStmt, context);
}
private void handleSetRole() throws PrivilegeException, UserException {
SetRoleExecutor.execute((SetRoleStmt) parsedStmt, context);
}

View File

@ -36,6 +36,8 @@ package com.starrocks.rpc;
import com.google.common.base.Preconditions;
import com.starrocks.common.Config;
import com.starrocks.proto.ExecuteCommandRequestPB;
import com.starrocks.proto.ExecuteCommandResultPB;
import com.starrocks.proto.PCancelPlanFragmentRequest;
import com.starrocks.proto.PCancelPlanFragmentResult;
import com.starrocks.proto.PCollectQueryStatisticsResult;
@ -265,6 +267,18 @@ public class DataNodeServiceClient {
return resultFuture;
}
public Future<ExecuteCommandResultPB> executeCommand(TNetworkAddress address, ExecuteCommandRequestPB request)
throws RpcException {
try {
final PBackendService service = BrpcProxy.getBackendService(address);
return service.executeCommandAsync(request);
} catch (Throwable e) {
LOG.warn("execute command exception, address={}:{} command:{}",
address.getHostname(), address.getPort(), request.command, e);
throw new RpcException(address.hostname, e.getMessage());
}
}
private static class SingletonHolder {
private static final DataNodeServiceClient INSTANCE = new DataNodeServiceClient();
}

View File

@ -68,8 +68,7 @@ public interface PBackendService {
attachmentHandler = ThriftClientAttachmentHandler.class)
Future<PMVMaintenanceTaskResult> submitMVMaintenanceTaskAsync(PMVMaintenanceTaskRequest request);
@ProtobufRPC(serviceName = "PBackendService", methodName = "execute_command", onceTalkTimeout = 60000,
attachmentHandler = ThriftClientAttachmentHandler.class)
@ProtobufRPC(serviceName = "PBackendService", methodName = "execute_command", onceTalkTimeout = 60000)
Future<ExecuteCommandResultPB> executeCommandAsync(ExecuteCommandRequestPB request);
}

View File

@ -571,7 +571,6 @@ public class Analyzer {
return null;
}
// ------------------------------------------- Cluster Management Statement ----------------------------------------
@Override

View File

@ -91,6 +91,7 @@ import com.starrocks.sql.ast.DropRoleStmt;
import com.starrocks.sql.ast.DropTableStmt;
import com.starrocks.sql.ast.DropUserStmt;
import com.starrocks.sql.ast.ExecuteAsStmt;
import com.starrocks.sql.ast.ExecuteScriptStmt;
import com.starrocks.sql.ast.ExportStmt;
import com.starrocks.sql.ast.InsertStmt;
import com.starrocks.sql.ast.InstallPluginStmt;
@ -772,6 +773,15 @@ public class PrivilegeChecker {
return null;
}
@Override
public Void visitExecuteScriptStatement(ExecuteScriptStmt statement, ConnectContext session) {
if (!GlobalStateMgr.getCurrentState().getAuth()
.checkGlobalPriv(session, PrivPredicate.ADMIN)) {
ErrorReport.reportSemanticException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "EXECUTE SCRIPT");
}
return null;
}
@Override
public Void visitShowMaterializedViewStatement(ShowMaterializedViewsStmt statement, ConnectContext session) {
String db = statement.getDb();

View File

@ -107,6 +107,7 @@ import com.starrocks.sql.ast.DropStatsStmt;
import com.starrocks.sql.ast.DropTableStmt;
import com.starrocks.sql.ast.DropUserStmt;
import com.starrocks.sql.ast.ExecuteAsStmt;
import com.starrocks.sql.ast.ExecuteScriptStmt;
import com.starrocks.sql.ast.ExportStmt;
import com.starrocks.sql.ast.GrantRoleStmt;
import com.starrocks.sql.ast.InsertStmt;
@ -1022,6 +1023,12 @@ public class PrivilegeCheckerV2 {
return null;
}
@Override
public Void visitExecuteScriptStatement(ExecuteScriptStmt statement, ConnectContext context) {
checkStmtOperatePrivilege(context);
return null;
}
@Override
public Void visitCreateRoleStatement(CreateRoleStmt statement, ConnectContext context) {
if (!PrivilegeActions.checkSystemAction(context, PrivilegeType.GRANT)) {

View File

@ -361,7 +361,6 @@ public abstract class AstVisitor<R, C> {
return visitShowStatement(statement, context);
}
// ------------------------------------------- Admin Statement -----------------------------------------------------
public R visitAdminSetConfigStatement(AdminSetConfigStmt statement, C context) {
@ -692,6 +691,10 @@ public abstract class AstVisitor<R, C> {
return visitStatement(statement, context);
}
public R visitExecuteScriptStatement(ExecuteScriptStmt statement, C context) {
return visitStatement(statement, context);
}
// --------------------------------------- Export Statement --------------------------------------------------------
public R visitExportStatement(ExportStmt statement, C context) {

View File

@ -0,0 +1,70 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// 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
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package com.starrocks.sql.ast;
import com.starrocks.analysis.RedirectStatus;
import com.starrocks.sql.parser.NodePosition;
// EXECUTE ON <BE_ID> <SCRIPT>
public class ExecuteScriptStmt extends StatementBase {
static long TIMEOUT_SEC_DEFAULT = 60;
long beId;
String script;
long timeoutSec = TIMEOUT_SEC_DEFAULT;
public ExecuteScriptStmt(long beId, String script) {
this(beId, script, NodePosition.ZERO);
}
public ExecuteScriptStmt(long beId, String script, NodePosition pos) {
super(pos);
this.beId = beId;
this.script = script;
}
public long getBeId() {
return beId;
}
public String getScript() {
return script;
}
public void setTimeoutSec(long timeoutSec) {
this.timeoutSec = timeoutSec;
}
public long getTimeoutSec() {
return timeoutSec;
}
@Override
public String toString() {
String s = String.format("EXECUTE ON %d %s", beId, script);
return s;
}
@Override
public <R, C> R accept(AstVisitor<R, C> visitor, C context) {
return visitor.visitExecuteScriptStatement(this, context);
}
@Override
public RedirectStatus getRedirectStatus() {
return RedirectStatus.NO_FORWARD;
}
}

View File

@ -198,6 +198,7 @@ import com.starrocks.sql.ast.DropWarehouseStmt;
import com.starrocks.sql.ast.EmptyStmt;
import com.starrocks.sql.ast.ExceptRelation;
import com.starrocks.sql.ast.ExecuteAsStmt;
import com.starrocks.sql.ast.ExecuteScriptStmt;
import com.starrocks.sql.ast.ExportStmt;
import com.starrocks.sql.ast.ExpressionPartitionDesc;
import com.starrocks.sql.ast.FunctionArgsDef;
@ -3135,6 +3136,14 @@ public class AstBuilder extends StarRocksBaseVisitor<ParseNode> {
return new SetWarehouseStmt(warehouseName, createPos(context));
}
@Override
public ParseNode visitExecuteScriptStatement(StarRocksParser.ExecuteScriptStatementContext context) {
long beId = Long.parseLong(context.INTEGER_VALUE().getText());
StringLiteral stringLiteral = (StringLiteral) visit(context.string());
String script = stringLiteral.getStringValue();
return new ExecuteScriptStmt(beId, script, createPos(context));
}
// ----------------------------------------------- Unsupported Statement -----------------------------------------------------
@Override

View File

@ -129,6 +129,7 @@ statement
| adminCheckTabletsStatement
| killStatement
| syncStatement
| executeScriptStatement
// Cluster Management Statement
| alterSystemStatement
@ -1515,6 +1516,10 @@ setWarehouseStatement
: SET WAREHOUSE identifierOrString
;
executeScriptStatement
: ADMIN EXECUTE ON INTEGER_VALUE string
;
unsupportedStatement
: START TRANSACTION (WITH CONSISTENT SNAPSHOT)?
| BEGIN WORK?

21
licenses/LICENSE-wren.txt Normal file
View File

@ -0,0 +1,21 @@
MIT License
Copyright (c) 2013-2021 Robert Nystrom and Wren Contributors
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.

View File

@ -0,0 +1,21 @@
MIT License
Copyright (c) 2019-2020 Matus Novak
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.