You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/01/27 16:59:07 UTC

[GitHub] [nifi-minifi-cpp] arpadboda opened a new pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

arpadboda opened a new pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716
 
 
   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically master)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374664831
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.h
 ##########
 @@ -56,21 +55,13 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
     db_ = NULL;
   }
 
-  // Destructor
-  virtual ~ProvenanceRepository() {
-    if (db_)
-      delete db_;
-  }
-
-  virtual void flush();
+  virtual void printStats();
 
 Review comment:
   Removed. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374664752
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.h
 ##########
 @@ -99,9 +90,26 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
     options.create_if_missing = true;
     options.use_direct_io_for_flush_and_compaction = true;
     options.use_direct_reads = true;
-    rocksdb::Status status = rocksdb::DB::Open(options, directory_, &db_);
+    int64_t max_buffer_size = 16 << 20;
 
 Review comment:
   Added comment. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r372356176
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.h
 ##########
 @@ -56,21 +55,13 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
     db_ = NULL;
   }
 
-  // Destructor
-  virtual ~ProvenanceRepository() {
-    if (db_)
-      delete db_;
-  }
-
-  virtual void flush();
+  virtual void printStats();
 
 Review comment:
   This new function is local to the class and doesn't need to be `virtual`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374670873
 
 

 ##########
 File path: libminifi/include/core/Repository.h
 ##########
 @@ -85,6 +85,10 @@ class Repository : public virtual core::SerializableComponent, public core::Trac
 
   virtual void flush();
 
+  virtual void printStats() {
+    return;
+  }
+
 
 Review comment:
   Wanted to remove and follow up in https://issues.apache.org/jira/browse/MINIFICPP-1145.
   
   Done. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375219496
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
 
 Review comment:
   The meaning of `count` and `size` are not clear from the prototype. How about `number_of_records` and `record_size`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375228737
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
 
 Review comment:
   The contract of this implementation implies that the caller is responsible for allocation by setting the size of the vector. In case the vector is returned an additional param would be needed to provide the size of the vector to return. 
   
   That could work as well, but in my opinion it was way more simple for the given use-case. 
   
   In nutshell: the current contract is just about filling the provided vector without taking care of allocation. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r372356539
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.h
 ##########
 @@ -99,9 +90,26 @@ class ProvenanceRepository : public core::Repository, public std::enable_shared_
     options.create_if_missing = true;
     options.use_direct_io_for_flush_and_compaction = true;
     options.use_direct_reads = true;
-    rocksdb::Status status = rocksdb::DB::Open(options, directory_, &db_);
+    int64_t max_buffer_size = 16 << 20;
 
 Review comment:
   What's the origin of this constant: `16 << 20`? Code comment would be appreciated.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375222314
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
+  for (int i = 0; i < count; ++i) {
+    std::vector<char> v(size);
+    generateData(v);
+    REQUIRE(repo.Put(std::to_string(i), reinterpret_cast<const uint8_t*>(v.data()), v.size()));
+  }
+}
+
+void verifyMaxKeyCount(const minifi::provenance::ProvenanceRepository& repo, uint64_t keyCount) {
+  uint64_t k = keyCount;
+
+  for (int i = 0; i < 5; ++i) {
+    std::this_thread::sleep_for(std::chrono::seconds(1));
 
 Review comment:
   Is 1s sleep necessary? I'm in favor of faster running tests if the effort is as little as reducing a delay and increasing the retry count.
   My suggestion: 50x iteration with 100ms sleep instead of 5x with 1s sleep

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374813056
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.cpp
 ##########
 @@ -17,72 +17,31 @@
  */
 
 #include "ProvenanceRepository.h"
-#include "rocksdb/write_batch.h"
 #include <string>
-#include <vector>
-#include "rocksdb/options.h"
-#include "provenance/Provenance.h"
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace provenance {
 
-void ProvenanceRepository::flush() {
-  rocksdb::WriteBatch batch;
-  std::string key;
-  std::string value;
-  rocksdb::ReadOptions options;
-  uint64_t decrement_total = 0;
-  while (keys_to_delete.size_approx() > 0) {
-    if (keys_to_delete.try_dequeue(key)) {
-      db_->Get(options, key, &value);
-      decrement_total += value.size();
-      batch.Delete(key);
-      logger_->log_debug("Removing %s", key);
-    }
-  }
-  if (db_->Write(rocksdb::WriteOptions(), &batch).ok()) {
-    logger_->log_debug("Decrementing %u from a repo size of %u", decrement_total, repo_size_.load());
-    if (decrement_total > repo_size_.load()) {
-      repo_size_ = 0;
-    } else {
-      repo_size_ -= decrement_total;
-    }
-  }
+void ProvenanceRepository::printStats() {
+  std::string key_count;
+  db_->GetProperty("rocksdb.estimate-num-keys", &key_count);
+
+  std::string table_readers;
+  db_->GetProperty("rocksdb.estimate-table-readers-mem", &table_readers);
+
+  std::string all_memtables;
+  db_->GetProperty("rocksdb.cur-size-all-mem-tables", &all_memtables);
+
+  logger_->log_info("Repository stats: key count: %zu, table readers size: %zu, all memory tables size: %zu",
+                    key_count, table_readers, all_memtables);
 }
 
 void ProvenanceRepository::run() {
   while (running_) {
-    std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
-    uint64_t curTime = getTimeMillis();
-    // threshold for purge
-    uint64_t purgeThreshold = max_partition_bytes_ * 3 / 4;
-
-    uint64_t size = getRepoSize();
-
-    if (size >= purgeThreshold) {
-      rocksdb::Iterator* it = db_->NewIterator(rocksdb::ReadOptions());
-      for (it->SeekToFirst(); it->Valid(); it->Next()) {
-        ProvenanceEventRecord eventRead;
-        std::string key = it->key().ToString();
-        uint64_t eventTime = eventRead.getEventTime(reinterpret_cast<uint8_t*>(const_cast<char*>(it->value().data())), it->value().size());
-        if (eventTime > 0) {
-          if ((curTime - eventTime) > (uint64_t)max_partition_millis_)
-            Delete(key);
-        } else {
-          logger_->log_debug("NiFi Provenance retrieve event %s fail", key);
-          Delete(key);
-        }
-      }
-      delete it;
-    }
-    flush();
-    size = getRepoSize();
-    if (size > (uint64_t)max_partition_bytes_)
-      repo_full_ = true;
-    else
-      repo_full_ = false;
+    std::this_thread::sleep_for(std::chrono::seconds(30));
 
 Review comment:
   Okay, added a temporary hack

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374678534
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.cpp
 ##########
 @@ -17,72 +17,31 @@
  */
 
 #include "ProvenanceRepository.h"
-#include "rocksdb/write_batch.h"
 #include <string>
-#include <vector>
-#include "rocksdb/options.h"
-#include "provenance/Provenance.h"
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace provenance {
 
-void ProvenanceRepository::flush() {
-  rocksdb::WriteBatch batch;
-  std::string key;
-  std::string value;
-  rocksdb::ReadOptions options;
-  uint64_t decrement_total = 0;
-  while (keys_to_delete.size_approx() > 0) {
-    if (keys_to_delete.try_dequeue(key)) {
-      db_->Get(options, key, &value);
-      decrement_total += value.size();
-      batch.Delete(key);
-      logger_->log_debug("Removing %s", key);
-    }
-  }
-  if (db_->Write(rocksdb::WriteOptions(), &batch).ok()) {
-    logger_->log_debug("Decrementing %u from a repo size of %u", decrement_total, repo_size_.load());
-    if (decrement_total > repo_size_.load()) {
-      repo_size_ = 0;
-    } else {
-      repo_size_ -= decrement_total;
-    }
-  }
+void ProvenanceRepository::printStats() {
+  std::string key_count;
+  db_->GetProperty("rocksdb.estimate-num-keys", &key_count);
+
+  std::string table_readers;
+  db_->GetProperty("rocksdb.estimate-table-readers-mem", &table_readers);
+
+  std::string all_memtables;
+  db_->GetProperty("rocksdb.cur-size-all-mem-tables", &all_memtables);
+
+  logger_->log_info("Repository stats: key count: %zu, table readers size: %zu, all memory tables size: %zu",
+                    key_count, table_readers, all_memtables);
 }
 
 void ProvenanceRepository::run() {
   while (running_) {
-    std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
-    uint64_t curTime = getTimeMillis();
-    // threshold for purge
-    uint64_t purgeThreshold = max_partition_bytes_ * 3 / 4;
-
-    uint64_t size = getRepoSize();
-
-    if (size >= purgeThreshold) {
-      rocksdb::Iterator* it = db_->NewIterator(rocksdb::ReadOptions());
-      for (it->SeekToFirst(); it->Valid(); it->Next()) {
-        ProvenanceEventRecord eventRead;
-        std::string key = it->key().ToString();
-        uint64_t eventTime = eventRead.getEventTime(reinterpret_cast<uint8_t*>(const_cast<char*>(it->value().data())), it->value().size());
-        if (eventTime > 0) {
-          if ((curTime - eventTime) > (uint64_t)max_partition_millis_)
-            Delete(key);
-        } else {
-          logger_->log_debug("NiFi Provenance retrieve event %s fail", key);
-          Delete(key);
-        }
-      }
-      delete it;
-    }
-    flush();
-    size = getRepoSize();
-    if (size > (uint64_t)max_partition_bytes_)
-      repo_full_ = true;
-    else
-      repo_full_ = false;
+    std::this_thread::sleep_for(std::chrono::seconds(30));
 
 Review comment:
   Yes, the same applied for previous implementation and still applies on average for 5 secs in FF repo (cleanup). 
   
   I also think that we should use callback timers, but I would prefer to do it in a unified way for all rocksdb repos in scope of https://issues.apache.org/jira/browse/MINIFICPP-1145

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375216809
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
+  for (int i = 0; i < count; ++i) {
+    std::vector<char> v(size);
+    generateData(v);
+    REQUIRE(repo.Put(std::to_string(i), reinterpret_cast<const uint8_t*>(v.data()), v.size()));
+  }
+}
+
+void verifyMaxKeyCount(const minifi::provenance::ProvenanceRepository& repo, uint64_t keyCount) {
+  uint64_t k = keyCount;
+
+  for (int i = 0; i < 5; ++i) {
+    std::this_thread::sleep_for(std::chrono::seconds(1));
+    k = std::min(k, repo.getKeyCount());
+    if (k < keyCount) {
+      break;
+    }
+  }
+
+  REQUIRE(k < keyCount);
+}
+
+TEST_CASE("Test size limit", "[sizeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
 
 Review comment:
   Nowhere, wrong comment, fixed. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374680995
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.cpp
 ##########
 @@ -17,72 +17,31 @@
  */
 
 #include "ProvenanceRepository.h"
-#include "rocksdb/write_batch.h"
 #include <string>
-#include <vector>
-#include "rocksdb/options.h"
-#include "provenance/Provenance.h"
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace provenance {
 
-void ProvenanceRepository::flush() {
-  rocksdb::WriteBatch batch;
-  std::string key;
-  std::string value;
-  rocksdb::ReadOptions options;
-  uint64_t decrement_total = 0;
-  while (keys_to_delete.size_approx() > 0) {
-    if (keys_to_delete.try_dequeue(key)) {
-      db_->Get(options, key, &value);
-      decrement_total += value.size();
-      batch.Delete(key);
-      logger_->log_debug("Removing %s", key);
-    }
-  }
-  if (db_->Write(rocksdb::WriteOptions(), &batch).ok()) {
-    logger_->log_debug("Decrementing %u from a repo size of %u", decrement_total, repo_size_.load());
-    if (decrement_total > repo_size_.load()) {
-      repo_size_ = 0;
-    } else {
-      repo_size_ -= decrement_total;
-    }
-  }
+void ProvenanceRepository::printStats() {
+  std::string key_count;
+  db_->GetProperty("rocksdb.estimate-num-keys", &key_count);
+
+  std::string table_readers;
+  db_->GetProperty("rocksdb.estimate-table-readers-mem", &table_readers);
+
+  std::string all_memtables;
+  db_->GetProperty("rocksdb.cur-size-all-mem-tables", &all_memtables);
+
+  logger_->log_info("Repository stats: key count: %zu, table readers size: %zu, all memory tables size: %zu",
+                    key_count, table_readers, all_memtables);
 }
 
 void ProvenanceRepository::run() {
   while (running_) {
-    std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
-    uint64_t curTime = getTimeMillis();
-    // threshold for purge
-    uint64_t purgeThreshold = max_partition_bytes_ * 3 / 4;
-
-    uint64_t size = getRepoSize();
-
-    if (size >= purgeThreshold) {
-      rocksdb::Iterator* it = db_->NewIterator(rocksdb::ReadOptions());
-      for (it->SeekToFirst(); it->Valid(); it->Next()) {
-        ProvenanceEventRecord eventRead;
-        std::string key = it->key().ToString();
-        uint64_t eventTime = eventRead.getEventTime(reinterpret_cast<uint8_t*>(const_cast<char*>(it->value().data())), it->value().size());
-        if (eventTime > 0) {
-          if ((curTime - eventTime) > (uint64_t)max_partition_millis_)
-            Delete(key);
-        } else {
-          logger_->log_debug("NiFi Provenance retrieve event %s fail", key);
-          Delete(key);
-        }
-      }
-      delete it;
-    }
-    flush();
-    size = getRepoSize();
-    if (size > (uint64_t)max_partition_bytes_)
-      repo_full_ = true;
-    else
-      repo_full_ = false;
+    std::this_thread::sleep_for(std::chrono::seconds(30));
 
 Review comment:
   I agree with moving as much as possible to a common implementation, but I don't want this delay on master even until that follow-up task is completed - it worsens the situation, by 3 times, if I understand correctly.
   I am fine with working this around for the time being by sleeping 30 times 1 second while checking `running_`, for example.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda closed pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda closed pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375230020
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
 
 Review comment:
   Done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375235671
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
+  for (int i = 0; i < count; ++i) {
+    std::vector<char> v(size);
+    generateData(v);
+    REQUIRE(repo.Put(std::to_string(i), reinterpret_cast<const uint8_t*>(v.data()), v.size()));
+  }
+}
+
+void verifyMaxKeyCount(const minifi::provenance::ProvenanceRepository& repo, uint64_t keyCount) {
+  uint64_t k = keyCount;
+
+  for (int i = 0; i < 5; ++i) {
+    std::this_thread::sleep_for(std::chrono::seconds(1));
+    k = std::min(k, repo.getKeyCount());
+    if (k < keyCount) {
+      break;
+    }
+  }
+
+  REQUIRE(k < keyCount);
+}
+
+TEST_CASE("Test size limit", "[sizeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
+      MAX_PROVENANCE_ENTRY_LIFE_TIME, TEST_PROVENANCE_STORAGE_SIZE, 1000);
+
+  auto configuration = std::make_shared<org::apache::nifi::minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_dbcontent_repository_directory_default, temp_dir);
+
+  REQUIRE(provdb.initialize(configuration));
+
+  uint64_t keyCount = 500;
+
+  provisionRepo(provdb, keyCount, 10240);
+
+  verifyMaxKeyCount(provdb, 200);
+}
+
+TEST_CASE("Test time limit", "[timeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
+                                                  TEST_PROVENANCE_ENTRY_LIFE_TIME, TEST_MAX_PROVENANCE_STORAGE_SIZE, 1000);
+
+  auto configuration = std::make_shared<org::apache::nifi::minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_dbcontent_repository_directory_default, temp_dir);
+
+  REQUIRE(provdb.initialize(configuration));
+
+  uint64_t keyCount = 500;
+
+  provisionRepo(provdb, keyCount / 2, 102400);
+
+  REQUIRE(provdb.getKeyCount() == 250);
+
+  /**
+   * Magic: TTL-based DB cleanup only triggers when writeBuffers are serialized to storage
+   * To achieve this 250 entries are put to DB with a total size that ensures at least one buffer is serialized
+   * Wait a sec to make sure the serialized records expire
 
 Review comment:
   Yes and no. 
   In theory, yes, on the other hand the smaller timeouts we use the higher the risk of becoming unstable. 
   200ms is something that can easily be overrun by a simple IO operation. For eg. assume parallel execution of a lot of testcases, there are many executing IO operations. 
   
   Changed the comment to reflect the code. 
   
   The execution of the two testcases together takes a bit more than 4 secs on my notebook. That means negligible additional time in case of parallel test execution (mostly because we have many testsuites taking 10+ secs). Without any visible gain I wouldn't risk lower timeouts and the effort spent on finetuning would simply be wasted. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374675202
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.cpp
 ##########
 @@ -17,72 +17,31 @@
  */
 
 #include "ProvenanceRepository.h"
-#include "rocksdb/write_batch.h"
 #include <string>
-#include <vector>
-#include "rocksdb/options.h"
-#include "provenance/Provenance.h"
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace provenance {
 
-void ProvenanceRepository::flush() {
-  rocksdb::WriteBatch batch;
-  std::string key;
-  std::string value;
-  rocksdb::ReadOptions options;
-  uint64_t decrement_total = 0;
-  while (keys_to_delete.size_approx() > 0) {
-    if (keys_to_delete.try_dequeue(key)) {
-      db_->Get(options, key, &value);
-      decrement_total += value.size();
-      batch.Delete(key);
-      logger_->log_debug("Removing %s", key);
-    }
-  }
-  if (db_->Write(rocksdb::WriteOptions(), &batch).ok()) {
-    logger_->log_debug("Decrementing %u from a repo size of %u", decrement_total, repo_size_.load());
-    if (decrement_total > repo_size_.load()) {
-      repo_size_ = 0;
-    } else {
-      repo_size_ -= decrement_total;
-    }
-  }
+void ProvenanceRepository::printStats() {
+  std::string key_count;
+  db_->GetProperty("rocksdb.estimate-num-keys", &key_count);
+
+  std::string table_readers;
+  db_->GetProperty("rocksdb.estimate-table-readers-mem", &table_readers);
+
+  std::string all_memtables;
+  db_->GetProperty("rocksdb.cur-size-all-mem-tables", &all_memtables);
+
+  logger_->log_info("Repository stats: key count: %zu, table readers size: %zu, all memory tables size: %zu",
+                    key_count, table_readers, all_memtables);
 }
 
 void ProvenanceRepository::run() {
   while (running_) {
-    std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
-    uint64_t curTime = getTimeMillis();
-    // threshold for purge
-    uint64_t purgeThreshold = max_partition_bytes_ * 3 / 4;
-
-    uint64_t size = getRepoSize();
-
-    if (size >= purgeThreshold) {
-      rocksdb::Iterator* it = db_->NewIterator(rocksdb::ReadOptions());
-      for (it->SeekToFirst(); it->Valid(); it->Next()) {
-        ProvenanceEventRecord eventRead;
-        std::string key = it->key().ToString();
-        uint64_t eventTime = eventRead.getEventTime(reinterpret_cast<uint8_t*>(const_cast<char*>(it->value().data())), it->value().size());
-        if (eventTime > 0) {
-          if ((curTime - eventTime) > (uint64_t)max_partition_millis_)
-            Delete(key);
-        } else {
-          logger_->log_debug("NiFi Provenance retrieve event %s fail", key);
-          Delete(key);
-        }
-      }
-      delete it;
-    }
-    flush();
-    size = getRepoSize();
-    if (size > (uint64_t)max_partition_bytes_)
-      repo_full_ = true;
-    else
-      repo_full_ = false;
+    std::this_thread::sleep_for(std::chrono::seconds(30));
 
 Review comment:
   This will prevent proper agent shutdown, on average for 15 seconds, which is pretty bad.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r372355656
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.cpp
 ##########
 @@ -17,72 +17,31 @@
  */
 
 #include "ProvenanceRepository.h"
-#include "rocksdb/write_batch.h"
 #include <string>
-#include <vector>
-#include "rocksdb/options.h"
-#include "provenance/Provenance.h"
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace provenance {
 
-void ProvenanceRepository::flush() {
-  rocksdb::WriteBatch batch;
-  std::string key;
-  std::string value;
-  rocksdb::ReadOptions options;
-  uint64_t decrement_total = 0;
-  while (keys_to_delete.size_approx() > 0) {
-    if (keys_to_delete.try_dequeue(key)) {
-      db_->Get(options, key, &value);
-      decrement_total += value.size();
-      batch.Delete(key);
-      logger_->log_debug("Removing %s", key);
-    }
-  }
-  if (db_->Write(rocksdb::WriteOptions(), &batch).ok()) {
-    logger_->log_debug("Decrementing %u from a repo size of %u", decrement_total, repo_size_.load());
-    if (decrement_total > repo_size_.load()) {
-      repo_size_ = 0;
-    } else {
-      repo_size_ -= decrement_total;
-    }
-  }
+void ProvenanceRepository::printStats() {
+  logger_->log_info("ProvenanceRepository stats:");
+
+  std::string out;
+  db_->GetProperty("rocksdb.estimate-num-keys", &out);
+  logger_->log_info("\\--Estimated key count: %s", out);
+
+  db_->GetProperty("rocksdb.estimate-table-readers-mem", &out);
+  logger_->log_info("\\--Estimated table readers memory consumption: %s", out);
+
+  db_->GetProperty("rocksdb.cur-size-all-mem-tables", &out);
+  logger_->log_info("\\--Size of all memory tables: %s", out);
 
 Review comment:
   It would greatly reduce log spam without reducing its usefulness if we logged all the stats in a single log message.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda edited a comment on issue #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda edited a comment on issue #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#issuecomment-581920115
 
 
   > It would be prudent to have some tests for at least the cleanup behaviour of the ProvenanceRepository, and it shouldn't be too hard to create.
   > 
   > Please also see my inline comment.
   
   If you provide a reasonable time to wait for after inserting the records to make sure rocksdb background tasks completed the cleanup I'm happy to do so. :)
   Although the "3 secs should be good" approach is not something I would follow, that's the best way to introduce unstable tests. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375228737
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
 
 Review comment:
   The contract of this implementation implies that the caller is responsible for allocation by setting the size of the vector. In case the vector is returned an additional param would be needed to provide the size of the vector to return. 
   
   That could work as well, but in my opinion it was way more simple for the given use-case. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375216482
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
+  for (int i = 0; i < count; ++i) {
+    std::vector<char> v(size);
+    generateData(v);
+    REQUIRE(repo.Put(std::to_string(i), reinterpret_cast<const uint8_t*>(v.data()), v.size()));
+  }
+}
+
+void verifyMaxKeyCount(const minifi::provenance::ProvenanceRepository& repo, uint64_t keyCount) {
+  uint64_t k = keyCount;
+
+  for (int i = 0; i < 5; ++i) {
+    std::this_thread::sleep_for(std::chrono::seconds(1));
+    k = std::min(k, repo.getKeyCount());
+    if (k < keyCount) {
+      break;
+    }
+  }
+
+  REQUIRE(k < keyCount);
+}
+
+TEST_CASE("Test size limit", "[sizeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
+      MAX_PROVENANCE_ENTRY_LIFE_TIME, TEST_PROVENANCE_STORAGE_SIZE, 1000);
+
+  auto configuration = std::make_shared<org::apache::nifi::minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_dbcontent_repository_directory_default, temp_dir);
+
+  REQUIRE(provdb.initialize(configuration));
+
+  uint64_t keyCount = 500;
+
+  provisionRepo(provdb, keyCount, 10240);
+
+  verifyMaxKeyCount(provdb, 200);
+}
+
+TEST_CASE("Test time limit", "[timeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
 
 Review comment:
   Copy-pasted my own comment that was obsolete even by the time I wrote it. :)
   Fixed. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375226770
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
+  for (int i = 0; i < count; ++i) {
+    std::vector<char> v(size);
+    generateData(v);
+    REQUIRE(repo.Put(std::to_string(i), reinterpret_cast<const uint8_t*>(v.data()), v.size()));
+  }
+}
+
+void verifyMaxKeyCount(const minifi::provenance::ProvenanceRepository& repo, uint64_t keyCount) {
+  uint64_t k = keyCount;
+
+  for (int i = 0; i < 5; ++i) {
+    std::this_thread::sleep_for(std::chrono::seconds(1));
+    k = std::min(k, repo.getKeyCount());
+    if (k < keyCount) {
+      break;
+    }
+  }
+
+  REQUIRE(k < keyCount);
+}
+
+TEST_CASE("Test size limit", "[sizeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
+      MAX_PROVENANCE_ENTRY_LIFE_TIME, TEST_PROVENANCE_STORAGE_SIZE, 1000);
+
+  auto configuration = std::make_shared<org::apache::nifi::minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_dbcontent_repository_directory_default, temp_dir);
+
+  REQUIRE(provdb.initialize(configuration));
+
+  uint64_t keyCount = 500;
+
+  provisionRepo(provdb, keyCount, 10240);
+
+  verifyMaxKeyCount(provdb, 200);
+}
+
+TEST_CASE("Test time limit", "[timeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
+                                                  TEST_PROVENANCE_ENTRY_LIFE_TIME, TEST_MAX_PROVENANCE_STORAGE_SIZE, 1000);
+
+  auto configuration = std::make_shared<org::apache::nifi::minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_dbcontent_repository_directory_default, temp_dir);
+
+  REQUIRE(provdb.initialize(configuration));
+
+  uint64_t keyCount = 500;
+
+  provisionRepo(provdb, keyCount / 2, 102400);
+
+  REQUIRE(provdb.getKeyCount() == 250);
+
+  /**
+   * Magic: TTL-based DB cleanup only triggers when writeBuffers are serialized to storage
+   * To achieve this 250 entries are put to DB with a total size that ensures at least one buffer is serialized
+   * Wait a sec to make sure the serialized records expire
 
 Review comment:
   "Wait a sec" but we wait 1500ms.
   Also, if we can reconfigure the cleanup logic to make the test run faster (i.e. not having to wait for 1500ms, but only like 200ms) without sacrificing stability, I'm strongly in favor of that. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375190678
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
+  for (int i = 0; i < count; ++i) {
+    std::vector<char> v(size);
+    generateData(v);
+    REQUIRE(repo.Put(std::to_string(i), reinterpret_cast<const uint8_t*>(v.data()), v.size()));
+  }
+}
+
+void verifyMaxKeyCount(const minifi::provenance::ProvenanceRepository& repo, uint64_t keyCount) {
+  uint64_t k = keyCount;
+
+  for (int i = 0; i < 5; ++i) {
+    std::this_thread::sleep_for(std::chrono::seconds(1));
+    k = std::min(k, repo.getKeyCount());
+    if (k < keyCount) {
+      break;
+    }
+  }
+
+  REQUIRE(k < keyCount);
+}
+
+TEST_CASE("Test size limit", "[sizeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
+      MAX_PROVENANCE_ENTRY_LIFE_TIME, TEST_PROVENANCE_STORAGE_SIZE, 1000);
+
+  auto configuration = std::make_shared<org::apache::nifi::minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_dbcontent_repository_directory_default, temp_dir);
+
+  REQUIRE(provdb.initialize(configuration));
+
+  uint64_t keyCount = 500;
+
+  provisionRepo(provdb, keyCount, 10240);
+
+  verifyMaxKeyCount(provdb, 200);
+}
+
+TEST_CASE("Test time limit", "[timeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
 
 Review comment:
   TEST_PROVENANCE_ENTRY_LIFE_TIME is `1 sec`, TEST_MAX_PROVENANCE_STORAGE_SIZE is `100 MB`, and I think in this case we expect to exceed the former, not the latter.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375217100
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
 
 Review comment:
   I prefer using return values over out params for returning values.
   My suggestion: `std::vector<char> generate_random_bytes(const size_t n)`
   
   I like the implementation, clean and beautiful.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375229974
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
+  for (int i = 0; i < count; ++i) {
+    std::vector<char> v(size);
+    generateData(v);
+    REQUIRE(repo.Put(std::to_string(i), reinterpret_cast<const uint8_t*>(v.data()), v.size()));
+  }
+}
+
+void verifyMaxKeyCount(const minifi::provenance::ProvenanceRepository& repo, uint64_t keyCount) {
+  uint64_t k = keyCount;
+
+  for (int i = 0; i < 5; ++i) {
+    std::this_thread::sleep_for(std::chrono::seconds(1));
 
 Review comment:
   Done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374664868
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.cpp
 ##########
 @@ -17,72 +17,31 @@
  */
 
 #include "ProvenanceRepository.h"
-#include "rocksdb/write_batch.h"
 #include <string>
-#include <vector>
-#include "rocksdb/options.h"
-#include "provenance/Provenance.h"
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace provenance {
 
-void ProvenanceRepository::flush() {
-  rocksdb::WriteBatch batch;
-  std::string key;
-  std::string value;
-  rocksdb::ReadOptions options;
-  uint64_t decrement_total = 0;
-  while (keys_to_delete.size_approx() > 0) {
-    if (keys_to_delete.try_dequeue(key)) {
-      db_->Get(options, key, &value);
-      decrement_total += value.size();
-      batch.Delete(key);
-      logger_->log_debug("Removing %s", key);
-    }
-  }
-  if (db_->Write(rocksdb::WriteOptions(), &batch).ok()) {
-    logger_->log_debug("Decrementing %u from a repo size of %u", decrement_total, repo_size_.load());
-    if (decrement_total > repo_size_.load()) {
-      repo_size_ = 0;
-    } else {
-      repo_size_ -= decrement_total;
-    }
-  }
+void ProvenanceRepository::printStats() {
+  logger_->log_info("ProvenanceRepository stats:");
+
+  std::string out;
+  db_->GetProperty("rocksdb.estimate-num-keys", &out);
+  logger_->log_info("\\--Estimated key count: %s", out);
+
+  db_->GetProperty("rocksdb.estimate-table-readers-mem", &out);
+  logger_->log_info("\\--Estimated table readers memory consumption: %s", out);
+
+  db_->GetProperty("rocksdb.cur-size-all-mem-tables", &out);
+  logger_->log_info("\\--Size of all memory tables: %s", out);
 
 Review comment:
   Done

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375190251
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
+  for (int i = 0; i < count; ++i) {
+    std::vector<char> v(size);
+    generateData(v);
+    REQUIRE(repo.Put(std::to_string(i), reinterpret_cast<const uint8_t*>(v.data()), v.size()));
+  }
+}
+
+void verifyMaxKeyCount(const minifi::provenance::ProvenanceRepository& repo, uint64_t keyCount) {
+  uint64_t k = keyCount;
+
+  for (int i = 0; i < 5; ++i) {
+    std::this_thread::sleep_for(std::chrono::seconds(1));
+    k = std::min(k, repo.getKeyCount());
+    if (k < keyCount) {
+      break;
+    }
+  }
+
+  REQUIRE(k < keyCount);
+}
+
+TEST_CASE("Test size limit", "[sizeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
 
 Review comment:
   MAX_PROVENANCE_ENTRY_LIFE_TIME is 60 seconds, where does `20 sec` come from?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374657478
 
 

 ##########
 File path: libminifi/include/core/Repository.h
 ##########
 @@ -85,6 +85,10 @@ class Repository : public virtual core::SerializableComponent, public core::Trac
 
   virtual void flush();
 
+  virtual void printStats() {
+    return;
+  }
+
 
 Review comment:
   Introducing an empty function to a base class is a smell. In this case, `printStats` doesn't need to be `virtual` or even present in `Repository`.
   
   From the standpoint of contracts:
   By introducing `printStats` to `Repository`, `Repository` now looks like something that can print stats of itself. The reality is that only `ProvenanceRepository` (and potentially later a select other few derived classes) can offer this functionality. In all other cases, this is a broken promise.
   
   From the OOP standpoint:
   `printStats` is a method specific to `ProvenanceRepository` but has leaked to `Repository`, breaking encapsulation.
   
   From a performance standpoint:
   Why pay for virtual dispatch when in all of the callers of `printStats`, the static and the dynamic types of the object match? We could even go as far as removing the function declaration from the headers altogether and giving it `static` linkage, therefore giving the compiler the freedom to throw the implementation away (except for 1 inlined instance) to reduce code bloat.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on issue #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
bakaid commented on issue #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#issuecomment-581921307
 
 
   > > It would be prudent to have some tests for at least the cleanup behaviour of the ProvenanceRepository, and it shouldn't be too hard to create.
   > > Please also see my inline comment.
   > 
   > If you provide a reasonable time to wait for after inserting the records to make ensure rocksdb background tasks completed the cleanup I'm happy to do so. :)
   > Although the "3 secs should be good" approach is not something I would follow, that's the best way to introduce unstable tests.
   
   I would wait in a loop with some delay, until it is cleaned up. If it takes too long, I would look into tweaking those background task times. If we have the chance - and now we have the chance - we shouldn't replace one untested, broken implementation with an another untested implementation that may be broken or could be broken later because we have no tests for it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
bakaid commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r374680995
 
 

 ##########
 File path: extensions/rocksdb-repos/ProvenanceRepository.cpp
 ##########
 @@ -17,72 +17,31 @@
  */
 
 #include "ProvenanceRepository.h"
-#include "rocksdb/write_batch.h"
 #include <string>
-#include <vector>
-#include "rocksdb/options.h"
-#include "provenance/Provenance.h"
 namespace org {
 namespace apache {
 namespace nifi {
 namespace minifi {
 namespace provenance {
 
-void ProvenanceRepository::flush() {
-  rocksdb::WriteBatch batch;
-  std::string key;
-  std::string value;
-  rocksdb::ReadOptions options;
-  uint64_t decrement_total = 0;
-  while (keys_to_delete.size_approx() > 0) {
-    if (keys_to_delete.try_dequeue(key)) {
-      db_->Get(options, key, &value);
-      decrement_total += value.size();
-      batch.Delete(key);
-      logger_->log_debug("Removing %s", key);
-    }
-  }
-  if (db_->Write(rocksdb::WriteOptions(), &batch).ok()) {
-    logger_->log_debug("Decrementing %u from a repo size of %u", decrement_total, repo_size_.load());
-    if (decrement_total > repo_size_.load()) {
-      repo_size_ = 0;
-    } else {
-      repo_size_ -= decrement_total;
-    }
-  }
+void ProvenanceRepository::printStats() {
+  std::string key_count;
+  db_->GetProperty("rocksdb.estimate-num-keys", &key_count);
+
+  std::string table_readers;
+  db_->GetProperty("rocksdb.estimate-table-readers-mem", &table_readers);
+
+  std::string all_memtables;
+  db_->GetProperty("rocksdb.cur-size-all-mem-tables", &all_memtables);
+
+  logger_->log_info("Repository stats: key count: %zu, table readers size: %zu, all memory tables size: %zu",
+                    key_count, table_readers, all_memtables);
 }
 
 void ProvenanceRepository::run() {
   while (running_) {
-    std::this_thread::sleep_for(std::chrono::milliseconds(purge_period_));
-    uint64_t curTime = getTimeMillis();
-    // threshold for purge
-    uint64_t purgeThreshold = max_partition_bytes_ * 3 / 4;
-
-    uint64_t size = getRepoSize();
-
-    if (size >= purgeThreshold) {
-      rocksdb::Iterator* it = db_->NewIterator(rocksdb::ReadOptions());
-      for (it->SeekToFirst(); it->Valid(); it->Next()) {
-        ProvenanceEventRecord eventRead;
-        std::string key = it->key().ToString();
-        uint64_t eventTime = eventRead.getEventTime(reinterpret_cast<uint8_t*>(const_cast<char*>(it->value().data())), it->value().size());
-        if (eventTime > 0) {
-          if ((curTime - eventTime) > (uint64_t)max_partition_millis_)
-            Delete(key);
-        } else {
-          logger_->log_debug("NiFi Provenance retrieve event %s fail", key);
-          Delete(key);
-        }
-      }
-      delete it;
-    }
-    flush();
-    size = getRepoSize();
-    if (size > (uint64_t)max_partition_bytes_)
-      repo_full_ = true;
-    else
-      repo_full_ = false;
+    std::this_thread::sleep_for(std::chrono::seconds(30));
 
 Review comment:
   I agree with moving as much as possible to a common implementation, but I don't want this delay on master even until that follow-up task is completed - it worsens the situation, by 3 times, if I understand correctly.
   I am fine with working this around for the time being by sleeping 30 times 1 seconds while checking `running_`, for example.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375217100
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
 
 Review comment:
   I prefer using return values over out params for returning values. They do it differently in C because C folks don't have exceptions to signal errors.
   My suggestion: `std::vector<char> generate_random_bytes(const size_t n)`
   
   I like the implementation, clean and beautiful.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on a change in pull request #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#discussion_r375235671
 
 

 ##########
 File path: libminifi/test/rocksdb-tests/DBProvenanceRepositoryTests.cpp
 ##########
 @@ -0,0 +1,122 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 "ProvenanceRepository.h"
+#include "../TestBase.h"
+#include <array>
+#include <chrono>
+#include <vector>
+#include <random>
+
+#define TEST_PROVENANCE_STORAGE_SIZE (1024*100)  // 100 KB
+#define TEST_MAX_PROVENANCE_STORAGE_SIZE (100*1024*1024)  // 100 MB
+
+#define TEST_PROVENANCE_ENTRY_LIFE_TIME (1000)  // 1 sec
+
+void generateData(std::vector<char>& data) {
+  std::random_device rd;
+  std::mt19937 eng(rd());
+
+  std::uniform_int_distribution<> distr(std::numeric_limits<char>::min(), std::numeric_limits<char>::max());
+  auto rand = std::bind(distr, eng);
+  std::generate_n(data.begin(), data.size(), rand);
+}
+
+void provisionRepo(minifi::provenance::ProvenanceRepository& repo, size_t count, size_t size) {
+  for (int i = 0; i < count; ++i) {
+    std::vector<char> v(size);
+    generateData(v);
+    REQUIRE(repo.Put(std::to_string(i), reinterpret_cast<const uint8_t*>(v.data()), v.size()));
+  }
+}
+
+void verifyMaxKeyCount(const minifi::provenance::ProvenanceRepository& repo, uint64_t keyCount) {
+  uint64_t k = keyCount;
+
+  for (int i = 0; i < 5; ++i) {
+    std::this_thread::sleep_for(std::chrono::seconds(1));
+    k = std::min(k, repo.getKeyCount());
+    if (k < keyCount) {
+      break;
+    }
+  }
+
+  REQUIRE(k < keyCount);
+}
+
+TEST_CASE("Test size limit", "[sizeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
+      MAX_PROVENANCE_ENTRY_LIFE_TIME, TEST_PROVENANCE_STORAGE_SIZE, 1000);
+
+  auto configuration = std::make_shared<org::apache::nifi::minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_dbcontent_repository_directory_default, temp_dir);
+
+  REQUIRE(provdb.initialize(configuration));
+
+  uint64_t keyCount = 500;
+
+  provisionRepo(provdb, keyCount, 10240);
+
+  verifyMaxKeyCount(provdb, 200);
+}
+
+TEST_CASE("Test time limit", "[timeLimitTest]") {
+  TestController testController;
+
+  char dirtemplate[] = "/tmp/db.XXXXXX";
+  auto temp_dir = testController.createTempDirectory(dirtemplate);
+  REQUIRE(!temp_dir.empty());
+
+  // 20 sec, 100kb - going to exceed the latter
+  minifi::provenance::ProvenanceRepository provdb("TestProvRepo", temp_dir,
+                                                  TEST_PROVENANCE_ENTRY_LIFE_TIME, TEST_MAX_PROVENANCE_STORAGE_SIZE, 1000);
+
+  auto configuration = std::make_shared<org::apache::nifi::minifi::Configure>();
+  configuration->set(minifi::Configure::nifi_dbcontent_repository_directory_default, temp_dir);
+
+  REQUIRE(provdb.initialize(configuration));
+
+  uint64_t keyCount = 500;
+
+  provisionRepo(provdb, keyCount / 2, 102400);
+
+  REQUIRE(provdb.getKeyCount() == 250);
+
+  /**
+   * Magic: TTL-based DB cleanup only triggers when writeBuffers are serialized to storage
+   * To achieve this 250 entries are put to DB with a total size that ensures at least one buffer is serialized
+   * Wait a sec to make sure the serialized records expire
 
 Review comment:
   Yes and no. 
   In theory, yes, on the other hand the smaller timeouts we use the higher the risk of becoming unstable. 
   200ms is something that can easily be overrun by a simple IO operation. For eg. assume parallel execution of a lot of testcases, there are many executing IO operations. 
   
   Changed the comment to reflect the code. 
   
   The execution of the two testcases together takes a bit more than 4 secs on my notebook. That means negligible additional time in case of parallel test execution (mostly because we have many testsuites taking 10+ secs). Without any visible gain I wouldn't risk lower timeouts and the effort wasted on finetuning would simply be wasted. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on issue #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on issue #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#issuecomment-582022406
 
 
   > > > It would be prudent to have some tests for at least the cleanup behaviour of the ProvenanceRepository, and it shouldn't be too hard to create.
   > > > Please also see my inline comment.
   > > 
   > > 
   > > If you provide a reasonable time to wait for after inserting the records to make ensure rocksdb background tasks completed the cleanup I'm happy to do so. :)
   > > Although the "3 secs should be good" approach is not something I would follow, that's the best way to introduce unstable tests.
   > 
   > I would wait in a loop with some delay, until it is cleaned up. If it takes too long, I would look into tweaking those background task times. If we have the chance - and now we have the chance - we shouldn't replace one untested, broken implementation with an another untested implementation that may be broken or could be broken later because we have no tests for it.
   
   Added tests, let's hope these are going to be stable on our CI as well. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi-minifi-cpp] arpadboda commented on issue #716: MINIFICPP-1127 - Provenance repo performance should be improved

Posted by GitBox <gi...@apache.org>.
arpadboda commented on issue #716: MINIFICPP-1127 - Provenance repo performance should be improved
URL: https://github.com/apache/nifi-minifi-cpp/pull/716#issuecomment-581920115
 
 
   > It would be prudent to have some tests for at least the cleanup behaviour of the ProvenanceRepository, and it shouldn't be too hard to create.
   > 
   > Please also see my inline comment.
   
   If you provide a reasonable time to wait for after inserting the records to make ensure rocksdb background tasks completed the cleanup I'm happy to do so. :)
   Although the "3 secs should be good" approach is not something I would follow, that's the best way to introduce unstable tests. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services