diff --git a/userspace/libsinsp/state/table_adapters.h b/userspace/libsinsp/state/table_adapters.h new file mode 100644 index 0000000000..98962bacfd --- /dev/null +++ b/userspace/libsinsp/state/table_adapters.h @@ -0,0 +1,263 @@ +// SPDX-License-Identifier: Apache-2.0 +/* +Copyright (C) 2024 The Falco Authors. +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +#pragma once + +#include + +namespace libsinsp +{ +namespace state +{ + +/** + * @brief A subclass of dynamic_struct::field_infos that have a fixed, + * and immutable, list of dynamic field definitions all declared at + * construction-time + */ +class fixed_dynamic_fields_infos : public dynamic_struct::field_infos +{ +public: + virtual ~fixed_dynamic_fields_infos() = default; + + inline fixed_dynamic_fields_infos(std::initializer_list infos): + field_infos(infos.begin()->defs_id()) + { + auto defs_id = infos.begin()->defs_id(); + for(const auto& f : infos) + { + if(f.defs_id() != defs_id) + { + throw sinsp_exception( + "inconsistent definition ID passed to fixed_dynamic_fields_infos"); + } + field_infos::add_field_info(f); + } + } + +protected: + const dynamic_struct::field_info& add_field_info(const dynamic_struct::field_info& field) override final + { + throw sinsp_exception("can't add field to fixed_dynamic_fields_infos: " + field.name()); + } +}; + +/** + * @brief An adapter for the libsinsp::state::table_entry interface + * that wraps a non-owning pointer of arbitrary type T. The underlying pointer + * can be set and unset arbitrarily, making this wrapper suitable for optimized + * allocations. Instances of table_entry from this adapter have no static fields, + * and make the wrapped value available as a single dynamic field. The dynamic + * fields definitions of this wrapper are fixed and immutable. + */ +template class value_table_entry_adapter : public libsinsp::state::table_entry +{ +public: + // note: this dynamic definitions are fixed in size and structure, + // so there's no need of worrying about specific identifier checks + // as they should be safely interchangeable + static const constexpr uintptr_t s_dynamic_fields_id = 1234; + + struct dynamic_fields_t : public fixed_dynamic_fields_infos + { + using _dfi = dynamic_struct::field_info; + + inline dynamic_fields_t(): fixed_dynamic_fields_infos({_dfi::build("value", 0, s_dynamic_fields_id)}) + { + } + + virtual ~dynamic_fields_t() = default; + }; + + inline explicit value_table_entry_adapter(): table_entry(nullptr), m_value(nullptr) {} + + virtual ~value_table_entry_adapter() = default; + + inline T* value() { return m_value; } + + inline const T* value() const { return m_value; } + + inline void set_value(T* v) { m_value = v; } + +protected: + virtual void get_dynamic_field(const dynamic_struct::field_info& i, void* out) override final + { + if(i.index() != 0 || i.defs_id() != s_dynamic_fields_id) + { + throw sinsp_exception( + "invalid field info passed to value_table_entry_adapter::get_dynamic_field"); + } + + if(i.info().index() == typeinfo::index_t::TI_STRING) + { + *((const char**)out) = ((const std::string*)m_value)->c_str(); + } + else + { + memcpy(out, (const void*)m_value, i.info().size()); + } + } + + virtual void set_dynamic_field(const dynamic_struct::field_info& i, const void* in) override final + { + if(i.index() != 0 || i.defs_id() != s_dynamic_fields_id) + { + throw sinsp_exception( + "invalid field info passed to value_table_entry_adapter::set_dynamic_field"); + } + + if(i.info().index() == typeinfo::index_t::TI_STRING) + { + *((std::string*)m_value) = *((const char**)in); + } + else + { + memcpy((void*)m_value, in, i.info().size()); + } + } + +private: + T* m_value; +}; + +/** + * @brief A template that helps converting STL container types (e.g. + * std::vector, std::list, etc) into tables compatible with the libsinsp + * state API. + * + * In this context, array-like types are mapped as tables with an uint64_t key + * representing the index of the element in the array -- as such, users should + * be extra careful when performing addition or deletion operations, as that + * can lead to expensive sparse array operations or results. + */ +template, + typename DynFields = typename TWrap::dynamic_fields_t> +class stl_container_table_adapter : public libsinsp::state::table +{ +public: + stl_container_table_adapter(const std::string& name, T& container): + table(name, _static_fields()), m_container(container) + { + set_dynamic_fields(std::make_shared()); + } + + virtual ~stl_container_table_adapter() = default; + + size_t entries_count() const override { return m_container.size(); } + + void clear_entries() override { m_container.clear(); } + + std::unique_ptr new_entry() const override + { + auto ret = std::make_unique(); + ret->set_dynamic_fields(this->dynamic_fields()); + return ret; + } + + bool foreach_entry(std::function pred) override + { + TWrap w; + w.set_dynamic_fields(this->dynamic_fields()); + for(auto& v : m_container) + { + w.set_value(&v); + if(!pred(w)) + { + return false; + } + } + return true; + } + + std::shared_ptr get_entry(const uint64_t& key) override + { + if(key >= m_container.size()) + { + return nullptr; + } + return wrap_value(&m_container[key]); + } + + std::shared_ptr + add_entry(const uint64_t& key, std::unique_ptr entry) override + { + if(!entry) + { + throw sinsp_exception("null entry added to table: " + this->name()); + } + if(entry->dynamic_fields() != this->dynamic_fields()) + { + throw sinsp_exception("entry with mismatching dynamic fields added to table: " + this->name()); + } + + auto value = dynamic_cast(entry.get()); + if(!value) + { + throw sinsp_exception("entry with mismatching type added to table: " + this->name()); + } + if(value->value() != nullptr) + { + throw sinsp_exception("entry with unexpected owned value added to table: " + this->name()); + } + + m_container.resize(key + 1); + return wrap_value(&m_container[key]); + } + + bool erase_entry(const uint64_t& key) override + { + if(key >= m_container.size()) + { + return false; + } + m_container.erase(m_container.begin() + key); + return true; + } + +private: + static inline const static_struct::field_infos* _static_fields() + { + static const auto s_fields = TWrap{}.static_fields(); + return &s_fields; + } + + static inline void wrap_deleter(TWrap* v) { v->set_value(nullptr); } + + // helps us dynamically allocate a batch of wrappers, creating new ones + // only if we need them. Wrappers are reused for multiple entries, and + // we leverage shared_ptrs to automatically release them once not anymore used + inline std::shared_ptr wrap_value(typename T::value_type* v) + { + for(auto& w : m_wrappers) + { + if(w.value() == nullptr) + { + w.set_value(v); + return std::shared_ptr(&w, wrap_deleter); + } + } + + // no wrapper is free among the allocated ones so add an extra one + auto& w = m_wrappers.emplace_back(); + w.set_value(v); + w.set_dynamic_fields(this->dynamic_fields()); + return std::shared_ptr(&w, wrap_deleter); + } + + T& m_container; + std::list m_wrappers; // using lists for ptr stability +}; + +}; // namespace state +}; // namespace libsinsp diff --git a/userspace/libsinsp/test/plugins.ut.cpp b/userspace/libsinsp/test/plugins.ut.cpp index 29c9b996ed..4a2a87bc79 100644 --- a/userspace/libsinsp/test/plugins.ut.cpp +++ b/userspace/libsinsp/test/plugins.ut.cpp @@ -758,6 +758,81 @@ TEST_F(sinsp_with_test_input, plugin_subtables) ASSERT_EQ(subtable->entries_count(), 0); } +TEST_F(sinsp_with_test_input, plugin_subtables_array) +{ + const constexpr auto num_entries_from_plugin = 10; + + auto& reg = m_inspector.get_table_registry(); + + register_plugin(&m_inspector, get_plugin_api_sample_syscall_subtables_array); + + auto table = reg->get_table("threads"); + ASSERT_NE(table, nullptr); + ASSERT_EQ(table->name(), "threads"); + ASSERT_EQ(table->entries_count(), 0); + ASSERT_EQ(table->key_info(), libsinsp::state::typeinfo::of()); + ASSERT_EQ(table->dynamic_fields()->fields().size(), 0); + + auto field = table->static_fields()->find("env_vars"); + ASSERT_NE(field, table->static_fields()->end()); + ASSERT_EQ(field->second.readonly(), true); + ASSERT_EQ(field->second.valid(), true); + ASSERT_EQ(field->second.name(), "env_vars"); + ASSERT_EQ(field->second.info(), libsinsp::state::typeinfo::of()); + + ASSERT_EQ(table->entries_count(), 0); + + // add a new entry to the thread table + ASSERT_NE(table->add_entry(5, table->new_entry()), nullptr); + auto entry = table->get_entry(5); + ASSERT_NE(entry, nullptr); + ASSERT_EQ(table->entries_count(), 1); + + // obtain a pointer to the subtable (check typing too) + auto subtable_acc = field->second.new_accessor(); + auto subtable = dynamic_cast>*>( + entry->get_static_field(subtable_acc)); + ASSERT_NE(subtable, nullptr); + ASSERT_EQ(subtable->name(), "env_vars"); + ASSERT_EQ(subtable->entries_count(), 0); + + // get an accessor to a dynamic field representing the array's values + ASSERT_EQ(subtable->dynamic_fields()->fields().size(), 1); + const auto& dfield = subtable->dynamic_fields()->fields().find("value"); + ASSERT_NE(dfield, table->dynamic_fields()->fields().end()); + ASSERT_EQ(dfield->second.readonly(), false); + ASSERT_EQ(dfield->second.valid(), true); + ASSERT_EQ(dfield->second.name(), "value"); + ASSERT_EQ(dfield->second.info(), libsinsp::state::typeinfo::of()); + auto dfieldacc = dfield->second.new_accessor(); + + // start the event capture + // we coordinate with the plugin by sending open events: for each one received, + // the plugin will take a subsequent action on which we then assert the status + open_inspector(); + + // step #0: the plugin should populate the fdtable + add_event_advance_ts(increasing_ts(), 0, PPME_SYSCALL_OPEN_E, 3, "/tmp/the_file", PPM_O_RDWR, 0); + ASSERT_EQ(subtable->entries_count(), num_entries_from_plugin); + + auto itt = [&](libsinsp::state::table_entry& e) -> bool + { + std::string tmpstr; + e.get_dynamic_field(dfieldacc, tmpstr); + EXPECT_EQ(tmpstr, "hello"); + return true; + }; + ASSERT_TRUE(subtable->foreach_entry(itt)); + + // step #1: the plugin should remove one entry from the fdtable + add_event_advance_ts(increasing_ts(), 0, PPME_SYSCALL_OPEN_E, 3, "/tmp/the_file", PPM_O_RDWR, 0); + ASSERT_EQ(subtable->entries_count(), num_entries_from_plugin - 1); + + // step #2: the plugin should cleae the fdtable + add_event_advance_ts(increasing_ts(), 0, PPME_SYSCALL_OPEN_E, 3, "/tmp/the_file", PPM_O_RDWR, 0); + ASSERT_EQ(subtable->entries_count(), 0); +} + // Scenario: we load a plugin expecting it to log // when it's initialized and destroyed. // We use a callback attached to the logger to assert the message. diff --git a/userspace/libsinsp/test/plugins/syscall_subtables_array.cpp b/userspace/libsinsp/test/plugins/syscall_subtables_array.cpp new file mode 100644 index 0000000000..afdf639687 --- /dev/null +++ b/userspace/libsinsp/test/plugins/syscall_subtables_array.cpp @@ -0,0 +1,281 @@ +// SPDX-License-Identifier: Apache-2.0 +/* +Copyright (C) 2024 The Falco Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +#include +#include +#include +#include +#include + +#include +#include "test_plugins.h" + +struct plugin_state +{ + std::string lasterr; + ss_plugin_table_t* thread_table; + + ss_plugin_table_field_t* table_field_envtable; + ss_plugin_table_field_t* table_field_envtable_value; + + uint8_t step = 0; +}; + +static const char* plugin_get_required_api_version() +{ + return PLUGIN_API_VERSION_STR; +} + +static const char* plugin_get_version() +{ + return "0.1.0"; +} + +static const char* plugin_get_name() +{ + return "sample_subtables_array"; +} + +static const char* plugin_get_description() +{ + return "some desc"; +} + +static const char* plugin_get_contact() +{ + return "some contact"; +} + +static const char* plugin_get_parse_event_sources() +{ + return "[\"syscall\"]"; +} + +static uint16_t* plugin_get_parse_event_types(uint32_t* num_types, ss_plugin_t* s) +{ + static uint16_t types[] = { PPME_SYSCALL_OPEN_E }; + *num_types = sizeof(types) / sizeof(uint16_t); + return &types[0]; +} + +static ss_plugin_t* plugin_init(const ss_plugin_init_input* in, ss_plugin_rc* rc) +{ + *rc = SS_PLUGIN_SUCCESS; + plugin_state *ret = new plugin_state(); + + if (!in || !in->tables) + { + *rc = SS_PLUGIN_FAILURE; + ret->lasterr = "invalid config input"; + return ret; + } + + // get an accessor to the threads table + ret->thread_table = in->tables->get_table( + in->owner, "threads", ss_plugin_state_type::SS_PLUGIN_ST_INT64); + if (!ret->thread_table) + { + *rc = SS_PLUGIN_FAILURE; + ret->lasterr = "can't access thread table"; + return ret; + } + + // get an accessor to the file descriptor tables owned by each thread info + ret->table_field_envtable = in->tables->fields.get_table_field( + ret->thread_table, "env_vars", ss_plugin_state_type::SS_PLUGIN_ST_TABLE); + if (!ret->table_field_envtable) + { + *rc = SS_PLUGIN_FAILURE; + ret->lasterr = "can't get envtable field in thread table"; + return ret; + } + + // create a new thread info -- the purpose is just to access its file + // descriptor table and obtain accessors for fields of that sub-table + auto entry = in->tables->writer_ext->create_table_entry(ret->thread_table); + if (!entry) + { + *rc = SS_PLUGIN_FAILURE; + ret->lasterr = "can't create subtable entry (init-time)"; + return ret; + } + + // read pointer to file descriptor table owned by the new thread info + ss_plugin_state_data data; + *rc = in->tables->reader_ext->read_entry_field(ret->thread_table, entry, ret->table_field_envtable, &data); + if (*rc != SS_PLUGIN_SUCCESS) + { + ret->lasterr = "can't read sub-table table entry field (init-time)"; + return ret; + } + auto envtable = data.table; + + // obtain accessor to one of the fields of file descriptor tables (name) + ret->table_field_envtable_value = in->tables->fields_ext->get_table_field( + envtable, "value", ss_plugin_state_type::SS_PLUGIN_ST_STRING); + if (!ret->table_field_envtable_value) + { + *rc = SS_PLUGIN_FAILURE; + ret->lasterr = "can't get sub-table 'value' field"; + return ret; + } + + // once we're done, destroy the temporarily-created thread info + in->tables->writer_ext->destroy_table_entry(ret->thread_table, entry); + + return ret; +} + +static void plugin_destroy(ss_plugin_t* s) +{ + delete ((plugin_state *) s); +} + +static const char* plugin_get_last_error(ss_plugin_t* s) +{ + return ((plugin_state *) s)->lasterr.c_str(); +} + +static ss_plugin_rc plugin_parse_event(ss_plugin_t *s, const ss_plugin_event_input *ev, const ss_plugin_event_parse_input* in) +{ + plugin_state *ps = (plugin_state *) s; + ss_plugin_state_data key; + ss_plugin_state_data out; + ss_plugin_state_data data; + + key.s64 = 0; + ss_plugin_table_entry_t* tinfo = in->table_reader_ext->get_table_entry(ps->thread_table, &key); + if (!tinfo) + { + ps->lasterr = "can't get table entry"; + return SS_PLUGIN_FAILURE; + } + + auto res = in->table_reader_ext->read_entry_field(ps->thread_table, tinfo, ps->table_field_envtable, &out); + if (res != SS_PLUGIN_SUCCESS) + { + ps->lasterr = "can't read table entry field"; + return SS_PLUGIN_FAILURE; + } + + ss_plugin_table_t* envtable = out.table; + + //add entries to the envtable + if(ps->step == 0) + { + int max_iterations = 10; + for (int i = 0; i < max_iterations; i++) + { + auto nentry = in->table_writer_ext->create_table_entry(envtable); + if (!nentry) + { + ps->lasterr = "can't create subtable entry"; + printf("ERR %s\n", ps->lasterr.c_str()); + return SS_PLUGIN_FAILURE; + } + key.s64 = i; + nentry = in->table_writer_ext->add_table_entry(envtable, &key, nentry); + if (!nentry) + { + ps->lasterr = "can't add subtable entry"; + printf("ERR %s\n", ps->lasterr.c_str()); + return SS_PLUGIN_FAILURE; + } + + data.str = "hello"; + res = in->table_reader_ext->read_entry_field(envtable, nentry, ps->table_field_envtable_value, &data); + if (res != SS_PLUGIN_SUCCESS) + { + ps->lasterr = "can't read subtable entry value field: " + std::string(in->get_owner_last_error(in->owner)); + printf("ERR %s\n", ps->lasterr.c_str()); + return SS_PLUGIN_FAILURE; + } + if (strcmp(data.str, "") != 0) + { + ps->lasterr = "wrong string read from subtable entry value field: " + std::string(data.str); + printf("ERR %s\n", ps->lasterr.c_str()); + return SS_PLUGIN_FAILURE; + } + + data.str = "hello"; + res = in->table_writer_ext->write_entry_field(envtable, nentry, ps->table_field_envtable_value, &data); + if (res != SS_PLUGIN_SUCCESS) + { + ps->lasterr = "can't write subtable entry value field: " + std::string(in->get_owner_last_error(in->owner)); + printf("ERR %s\n", ps->lasterr.c_str()); + return SS_PLUGIN_FAILURE; + } + + in->table_reader_ext->release_table_entry(envtable, nentry); + } + + ps->step++; + in->table_reader_ext->release_table_entry(ps->thread_table, tinfo); + return SS_PLUGIN_SUCCESS; + } + + // remove one entry from the envtable + if(ps->step == 1) + { + key.s64 = 0; + auto res = in->table_writer_ext->erase_table_entry(envtable, &key); + if (res != SS_PLUGIN_SUCCESS) + { + ps->lasterr = "can't erase subtable entry"; + printf("ERR %s\n", ps->lasterr.c_str()); + return SS_PLUGIN_FAILURE; + } + + ps->step++; + in->table_reader_ext->release_table_entry(ps->thread_table, tinfo); + return SS_PLUGIN_SUCCESS; + } + + // clear the envtable + if(ps->step == 2) + { + auto res = in->table_writer_ext->clear_table(envtable); + if (res != SS_PLUGIN_SUCCESS) + { + ps->lasterr = "can't clear subtable"; + printf("ERR %s\n", ps->lasterr.c_str()); + return SS_PLUGIN_FAILURE; + } + + ps->step++; + in->table_reader_ext->release_table_entry(ps->thread_table, tinfo); + return SS_PLUGIN_SUCCESS; + } + + return SS_PLUGIN_SUCCESS; +} + +void get_plugin_api_sample_syscall_subtables_array(plugin_api& out) +{ + memset(&out, 0, sizeof(plugin_api)); + out.get_required_api_version = plugin_get_required_api_version; + out.get_version = plugin_get_version; + out.get_description = plugin_get_description; + out.get_contact = plugin_get_contact; + out.get_name = plugin_get_name; + out.get_last_error = plugin_get_last_error; + out.init = plugin_init; + out.destroy = plugin_destroy; + out.get_parse_event_sources = plugin_get_parse_event_sources; + out.get_parse_event_types = plugin_get_parse_event_types; + out.parse_event = plugin_parse_event; +} diff --git a/userspace/libsinsp/test/plugins/test_plugins.h b/userspace/libsinsp/test/plugins/test_plugins.h index 00ce640fcf..f00c21044b 100644 --- a/userspace/libsinsp/test/plugins/test_plugins.h +++ b/userspace/libsinsp/test/plugins/test_plugins.h @@ -27,3 +27,4 @@ void get_plugin_api_sample_plugin_source(plugin_api& out); void get_plugin_api_sample_plugin_extract(plugin_api& out); void get_plugin_api_sample_syscall_tables(plugin_api& out); void get_plugin_api_sample_syscall_subtables(plugin_api& out); +void get_plugin_api_sample_syscall_subtables_array(plugin_api& out); diff --git a/userspace/libsinsp/test/state.ut.cpp b/userspace/libsinsp/test/state.ut.cpp index 781e256bdc..945a7ab1e3 100644 --- a/userspace/libsinsp/test/state.ut.cpp +++ b/userspace/libsinsp/test/state.ut.cpp @@ -308,7 +308,7 @@ TEST(thread_manager, table_access) { // note: used for regression checks, keep this updated as we make // new fields available - static const int s_threadinfo_static_fields_count = 25; + static const int s_threadinfo_static_fields_count = 27; sinsp inspector; auto table = static_cast*>(inspector.m_thread_manager.get()); @@ -543,3 +543,124 @@ TEST(thread_manager, fdtable_access) ASSERT_NO_THROW(subtable->clear_entries()); ASSERT_EQ(subtable->entries_count(), 0); } + +TEST(thread_manager, env_vars_access) +{ + sinsp inspector; + auto& reg = inspector.get_table_registry(); + + ASSERT_EQ(reg->tables().size(), 1); + ASSERT_NE(reg->tables().find("threads"), reg->tables().end()); + + auto table = reg->get_table("threads"); + EXPECT_EQ(table->name(), "threads"); + EXPECT_EQ(table->entries_count(), 0); + EXPECT_EQ(table->key_info(), libsinsp::state::typeinfo::of()); + EXPECT_EQ(table->dynamic_fields()->fields().size(), 0); + + auto field = table->static_fields()->find("env_vars"); + ASSERT_NE(field, table->static_fields()->end()); + EXPECT_EQ(field->second.readonly(), true); + EXPECT_EQ(field->second.valid(), true); + EXPECT_EQ(field->second.name(), "env_vars"); + EXPECT_EQ(field->second.info(), libsinsp::state::typeinfo::of()); + + ASSERT_EQ(table->entries_count(), 0); + + //add two new entries to the thread table + ASSERT_NE(table->add_entry(1, table->new_entry()), nullptr); + auto entry = table->get_entry(1); + ASSERT_NE(entry, nullptr); + ASSERT_EQ(table->entries_count(), 1); + + //getting the fd tables from the newly created threads + auto subtable_acc = field->second.new_accessor(); + auto subtable = dynamic_cast>*>(entry->get_static_field(subtable_acc)); + ASSERT_NE(subtable, nullptr); + EXPECT_EQ(subtable->name(), "env_vars"); + EXPECT_EQ(subtable->entries_count(), 0); + EXPECT_EQ(subtable->key_info(), libsinsp::state::typeinfo::of()); + EXPECT_EQ(subtable->static_fields()->size(), 0); + EXPECT_EQ(subtable->dynamic_fields()->fields().size(), 1); + + //getting an existing field + auto sfield = subtable->dynamic_fields()->fields().find("value"); + ASSERT_NE(sfield, subtable->dynamic_fields()->fields().end()); + EXPECT_EQ(sfield->second.readonly(), false); + EXPECT_EQ(sfield->second.valid(), true); + EXPECT_EQ(sfield->second.name(), "value"); + EXPECT_EQ(sfield->second.info(), libsinsp::state::typeinfo::of()); + + auto fieldacc = sfield->second.new_accessor(); + + // adding new entries to the subtable + uint64_t max_iterations = 10; + for (uint64_t i = 0; i < max_iterations; i++) + { + ASSERT_EQ(subtable->entries_count(), i); + + // get non-existing entry + ASSERT_EQ(subtable->get_entry(i), nullptr); + + // creating and adding a fd to the table + auto t = subtable->add_entry(i, subtable->new_entry()); + ASSERT_NE(t, nullptr); + ASSERT_NE(subtable->get_entry(i), nullptr); + ASSERT_EQ(subtable->entries_count(), i + 1); + + // read and write from newly-created entry + std::string tmpstr = "test"; + t->get_dynamic_field(fieldacc, tmpstr); + ASSERT_EQ(tmpstr, ""); + tmpstr = "hello"; + t->set_dynamic_field(fieldacc, tmpstr); + tmpstr = ""; + t->get_dynamic_field(fieldacc, tmpstr); + ASSERT_EQ(tmpstr, "hello"); + } + + // full iteration + auto it = [&](libsinsp::state::table_entry& e) -> bool + { + std::string tmpstr = "test"; + e.get_dynamic_field(fieldacc, tmpstr); + EXPECT_EQ(tmpstr, "hello"); + return true; + }; + ASSERT_TRUE(subtable->foreach_entry(it)); + + // iteration with break-out + ASSERT_FALSE(subtable->foreach_entry([&](libsinsp::state::table_entry& e) -> bool + { + return false; + })); + + // iteration with error + ASSERT_ANY_THROW(subtable->foreach_entry([&](libsinsp::state::table_entry& e) -> bool + { + throw sinsp_exception("some error"); + })); + + // erasing an unknown fd + ASSERT_EQ(subtable->erase_entry(max_iterations), false); + ASSERT_EQ(subtable->entries_count(), max_iterations); + + // erase one of the newly-created fd + ASSERT_EQ(subtable->erase_entry(0), true); + ASSERT_EQ(subtable->entries_count(), max_iterations - 1); + + // check that changes are reflected in thread's table + auto tinfo = inspector.m_thread_manager->get_thread_ref(1); + ASSERT_NE(tinfo, nullptr); + + ASSERT_EQ(tinfo->m_env.size(), max_iterations - 1); + for (const auto & v : tinfo->m_env) + { + EXPECT_EQ(v, "hello"); + } + + // clear all + ASSERT_NO_THROW(subtable->clear_entries()); + EXPECT_EQ(subtable->entries_count(), 0); + EXPECT_EQ(tinfo->m_env.size(), 0); +} diff --git a/userspace/libsinsp/threadinfo.cpp b/userspace/libsinsp/threadinfo.cpp index 0d0eba97aa..755e105f87 100644 --- a/userspace/libsinsp/threadinfo.cpp +++ b/userspace/libsinsp/threadinfo.cpp @@ -48,7 +48,9 @@ sinsp_threadinfo::sinsp_threadinfo(sinsp* inspector, std::shared_ptr #include #include -#include #include +#include +#include struct erase_fd_params { @@ -738,6 +739,8 @@ class SINSP_PUBLIC sinsp_threadinfo : public libsinsp::state::table_entry uint16_t m_lastevent_cpuid; sinsp_evt::category m_lastevent_category; bool m_parent_loop_detected; + libsinsp::state::stl_container_table_adapter m_args_table_adapter; + libsinsp::state::stl_container_table_adapter m_env_table_adapter; }; /*@}*/