From 96705d2143d350d42325c63f4bfed7caf14c156c Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Thu, 22 Feb 2024 21:26:06 +0100 Subject: [PATCH] [hotfix] Fix namespaces in flink compaction filter --- java/rocksjni/flink_compactionfilterjni.cc | 62 +++++++++++----------- 1 file changed, 32 insertions(+), 30 deletions(-) diff --git a/java/rocksjni/flink_compactionfilterjni.cc b/java/rocksjni/flink_compactionfilterjni.cc index 9f0954b43..a45f12d7b 100644 --- a/java/rocksjni/flink_compactionfilterjni.cc +++ b/java/rocksjni/flink_compactionfilterjni.cc @@ -12,8 +12,6 @@ #include "rocksjni/jnicallback.h" #include "utilities/flink/flink_compaction_filter.h" -using namespace ROCKSDB_NAMESPACE::flink; - class JniCallbackBase : public ROCKSDB_NAMESPACE::JniCallback { public: JniCallbackBase(JNIEnv* env, jobject jcallback_obj) @@ -94,7 +92,8 @@ class JavaListElemenFilterFactory assert(m_jcreate_filter_methodid != nullptr); } - FlinkCompactionFilter::ListElementFilter* CreateListElementFilter( + ROCKSDB_NAMESPACE::flink::FlinkCompactionFilter::ListElementFilter* + CreateListElementFilter( std::shared_ptr /*logger*/) const override { jboolean attached_thread = JNI_FALSE; JNIEnv* env = getJniEnv(&attached_thread); @@ -141,16 +140,16 @@ class JavaTimeProvider jmethodID m_jcurrent_timestamp_methodid; }; -static FlinkCompactionFilter::ListElementFilterFactory* -createListElementFilterFactory(JNIEnv* env, jint ji_list_elem_len, - jobject jlist_filter_factory) { - FlinkCompactionFilter::ListElementFilterFactory* list_filter_factory = - nullptr; +static ROCKSDB_NAMESPACE::flink::FlinkCompactionFilter:: + ListElementFilterFactory* + createListElementFilterFactory(JNIEnv* env, jint ji_list_elem_len, + jobject jlist_filter_factory) { + ROCKSDB_NAMESPACE::flink::FlinkCompactionFilter::ListElementFilterFactory* + list_filter_factory = nullptr; if (ji_list_elem_len > 0) { auto fixed_size = static_cast(ji_list_elem_len); - list_filter_factory = - new FlinkCompactionFilter::FixedListElementFilterFactory( - fixed_size, static_cast(0)); + list_filter_factory = new ROCKSDB_NAMESPACE::flink::FlinkCompactionFilter:: + FixedListElementFilterFactory(fixed_size, static_cast(0)); } else if (jlist_filter_factory != nullptr) { list_filter_factory = new JavaListElemenFilterFactory(env, jlist_filter_factory); @@ -165,10 +164,10 @@ createListElementFilterFactory(JNIEnv* env, jint ji_list_elem_len, */ jlong Java_org_rocksdb_FlinkCompactionFilter_createNewFlinkCompactionFilterConfigHolder( JNIEnv* /* env */, jclass /* jcls */) { - using namespace ROCKSDB_NAMESPACE::flink; return reinterpret_cast( - new std::shared_ptr( - new FlinkCompactionFilter::ConfigHolder())); + new std::shared_ptr< + ROCKSDB_NAMESPACE::flink::FlinkCompactionFilter::ConfigHolder>( + new ROCKSDB_NAMESPACE::flink::FlinkCompactionFilter::ConfigHolder())); } /* @@ -178,10 +177,8 @@ jlong Java_org_rocksdb_FlinkCompactionFilter_createNewFlinkCompactionFilterConfi */ void Java_org_rocksdb_FlinkCompactionFilter_disposeFlinkCompactionFilterConfigHolder( JNIEnv* /* env */, jclass /* jcls */, jlong handle) { - using namespace ROCKSDB_NAMESPACE::flink; - auto* config_holder = - reinterpret_cast*>( - handle); + auto* config_holder = reinterpret_cast*>(handle); delete config_holder; } @@ -193,9 +190,9 @@ void Java_org_rocksdb_FlinkCompactionFilter_disposeFlinkCompactionFilterConfigHo jlong Java_org_rocksdb_FlinkCompactionFilter_createNewFlinkCompactionFilter0( JNIEnv* env, jclass /* jcls */, jlong config_holder_handle, jobject jtime_provider, jlong logger_handle) { - using namespace ROCKSDB_NAMESPACE::flink; auto config_holder = - *(reinterpret_cast*>( + *(reinterpret_cast*>( config_holder_handle)); auto time_provider = new JavaTimeProvider(env, jtime_provider); auto logger = @@ -204,10 +201,13 @@ jlong Java_org_rocksdb_FlinkCompactionFilter_createNewFlinkCompactionFilter0( : *(reinterpret_cast< std::shared_ptr*>( logger_handle)); - return reinterpret_cast(new FlinkCompactionFilter( - config_holder, - std::unique_ptr(time_provider), - logger)); + return reinterpret_cast( + new ROCKSDB_NAMESPACE::flink::FlinkCompactionFilter( + config_holder, + std::unique_ptr< + ROCKSDB_NAMESPACE::flink::FlinkCompactionFilter::TimeProvider>( + time_provider), + logger)); } /* @@ -221,19 +221,21 @@ jboolean Java_org_rocksdb_FlinkCompactionFilter_configureFlinkCompactionFilter( jlong jquery_time_after_num_entries, jint ji_list_elem_len, jobject jlist_filter_factory) { auto state_type = - static_cast(ji_state_type); + static_cast( + ji_state_type); auto timestamp_offset = static_cast(ji_timestamp_offset); auto ttl = static_cast(jl_ttl_milli); auto query_time_after_num_entries = static_cast(jquery_time_after_num_entries); auto config_holder = - *(reinterpret_cast*>( + *(reinterpret_cast*>( handle)); auto list_filter_factory = createListElementFilterFactory( env, ji_list_elem_len, jlist_filter_factory); - auto config = new FlinkCompactionFilter::Config{ + auto config = new ROCKSDB_NAMESPACE::flink::FlinkCompactionFilter::Config{ state_type, timestamp_offset, ttl, query_time_after_num_entries, - std::unique_ptr( - list_filter_factory)}; + std::unique_ptr(list_filter_factory)}; return static_cast(config_holder->Configure(config)); -} \ No newline at end of file +}