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 2021/03/23 09:22:19 UTC

[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1004: MINIFICPP-1450 - Revive SQL processors

adamdebreceni commented on a change in pull request #1004:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1004#discussion_r599396392



##########
File path: extensions/sql/data/MaxCollector.h
##########
@@ -77,79 +74,66 @@ class MaxCollector: public SQLRowSubscriber {
   void processColumn(const std::string& name, const char* value) override {}
 
   template <typename T>
-  struct MaxValue {
-    void updateMaxValue(const std::string& name, const T& value) {
-      const auto it = mapColumnNameValue_.find(name);
-      if (it == mapColumnNameValue_.end()) {
-        mapColumnNameValue_.insert({ name, value });
+  class MaxValue {
+   public:
+    void updateMaxValue(const std::string& column, const T& value) {
+      const auto it = column_maxima.find(column);
+      if (it == column_maxima.end()) {
+        column_maxima.insert({ column, value });

Review comment:
       done

##########
File path: extensions/sql/processors/FlowFileSource.h
##########
@@ -0,0 +1,100 @@
+/**
+ * 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.
+ */
+
+#pragma once
+
+#include <string>
+#include <vector>
+#include <memory>
+
+#include "core/Property.h"
+#include "utils/Enum.h"
+#include "data/SQLRowsetProcessor.h"
+#include "ProcessSession.h"
+#include "data/JSONSQLWriter.h"
+
+namespace org {
+namespace apache {
+namespace nifi {
+namespace minifi {
+namespace processors {
+
+class FlowFileSource {
+ public:
+  static const std::string FRAGMENT_IDENTIFIER;
+  static const std::string FRAGMENT_COUNT;
+  static const std::string FRAGMENT_INDEX;
+
+  static const core::Property OutputFormat;
+  static const core::Property MaxRowsPerFlowFile;
+
+  SMART_ENUM(OutputType,
+    (JSON, "JSON"),
+    (JSONPretty, "JSON-Pretty")
+  )
+
+ protected:
+  class FlowFileGenerator : public sql::SQLRowSubscriber {
+   public:
+    FlowFileGenerator(core::ProcessSession& session, sql::JSONSQLWriter& json_writer)
+      : session_(session),
+        json_writer_(json_writer) {}
+
+    void beginProcessBatch() override {
+      current_batch_size_ = 0;
+    }
+    void endProcessBatch(Progress progress) override;
+    void beginProcessRow() override {}
+    void endProcessRow() override {
+      ++current_batch_size_;
+    }
+    void processColumnNames(const std::vector<std::string>& names) override {}
+    void processColumn(const std::string& name, const std::string& value) override {}
+    void processColumn(const std::string& name, double value) override {}
+    void processColumn(const std::string& name, int value) override {}
+    void processColumn(const std::string& name, long long value) override {}
+    void processColumn(const std::string& name, unsigned long long value) override {}
+    void processColumn(const std::string& name, const char* value) override {}
+
+    std::shared_ptr<core::FlowFile> getLastFlowFile() {

Review comment:
       done

##########
File path: extensions/sql/processors/QueryDatabaseTable.cpp
##########
@@ -75,361 +72,232 @@ const core::Property QueryDatabaseTable::s_maxValueColumnNames(
     "If no columns are provided, all rows from the table will be considered, which could have a performance impact. "
     "NOTE: It is important to use consistent max-value column names for a given table for incremental fetch to work properly. "
     "NOTE: Because of a limitation of database access library 'soci', which doesn't support milliseconds in it's 'dt_date', "
-    "there is a possibility that flowfiles might have duplicated records, if a max-value column with 'dt_date' type has value with milliseconds.")->
-    supportsExpressionLanguage(true)->build());
+    "there is a possibility that flowfiles might have duplicated records, if a max-value column with 'dt_date' type has value with milliseconds.")
+  ->supportsExpressionLanguage(true)->build());
 
-const core::Property QueryDatabaseTable::s_whereClause(
-  core::PropertyBuilder::createProperty("db-fetch-where-clause")->isRequired(false)->withDescription(
-    "A custom clause to be added in the WHERE condition when building SQL queries.")->supportsExpressionLanguage(true)->build());
+const core::Property QueryDatabaseTable::WhereClause(
+  core::PropertyBuilder::createProperty("Where Clause")
+  ->isRequired(false)
+  ->withDescription("A custom clause to be added in the WHERE condition when building SQL queries.")
+  ->supportsExpressionLanguage(true)->build());
 
-const core::Property QueryDatabaseTable::s_sqlQuery(
-  core::PropertyBuilder::createProperty("db-fetch-sql-query")->isRequired(false)->withDescription(
-    "A custom SQL query used to retrieve data. Instead of building a SQL query from other properties, this query will be wrapped as a sub-query. "
-    "Query must have no ORDER BY statement.")->supportsExpressionLanguage(true)->build());
+const std::string QueryDatabaseTable::InitialMaxValueDynamicPropertyPrefix("initial.maxvalue.");
 
-const core::Property QueryDatabaseTable::s_maxRowsPerFlowFile(
-  core::PropertyBuilder::createProperty("qdbt-max-rows")->isRequired(true)->withDefaultValue<int>(0)->withDescription(
-    "The maximum number of result rows that will be included in a single FlowFile. This will allow you to break up very large result sets into multiple FlowFiles. "
-    "If the value specified is zero, then all rows are returned in a single FlowFile.")->supportsExpressionLanguage(true)->build());
+const core::Relationship QueryDatabaseTable::Success("success", "Successfully created FlowFile from SQL query result set.");
 
-const core::Property QueryDatabaseTable::s_stateDirectory(
-  core::PropertyBuilder::createProperty("State Directory")->isRequired(false)->withDefaultValue("QDTState")->withDescription("DEPRECATED. Only use it for state migration from the state file, supplying the legacy state directory.")->build());
+const std::string QueryDatabaseTable::RESULT_TABLE_NAME = "tablename";
+const std::string QueryDatabaseTable::RESULT_ROW_COUNT = "querydbtable.row.count";
 
-const std::string QueryDatabaseTable::s_initialMaxValueDynamicPropertyPrefix("initial.maxvalue.");
+const std::string QueryDatabaseTable::TABLENAME_KEY = "tablename";
+const std::string QueryDatabaseTable::MAXVALUE_KEY_PREFIX = "maxvalue.";
 
-const core::Relationship QueryDatabaseTable::s_success("success", "Successfully created FlowFile from SQL query result set.");
-
-static const std::string ResultTableName = "tablename";
-static const std::string ResultRowCount = "querydbtable.row.count";
-
-static const std::string TABLENAME_KEY = "tablename";
-static const std::string MAXVALUE_KEY_PREFIX = "maxvalue.";
-
-// State
-class LegacyState {
- public:
-  LegacyState(const std::string& tableName, const std::string& stateDir, const std::string& uuid, std::shared_ptr<logging::Logger> logger)
-    :tableName_(tableName), logger_(logger) {
-
-    filePath_ = utils::file::FileUtils::concat_path(
-            utils::file::FileUtils::concat_path(
-              utils::file::FileUtils::concat_path(stateDir, "uuid"), uuid), "State.txt");
+// QueryDatabaseTable
+QueryDatabaseTable::QueryDatabaseTable(const std::string& name, utils::Identifier uuid)
+  : SQLProcessor(name, uuid, logging::LoggerFactory<QueryDatabaseTable>::getLogger()) {
+}
 
-    if (!getStateFromFile())
-      return;
+void QueryDatabaseTable::initialize() {
+  //! Set the supported properties
+  setSupportedProperties({
+    DBControllerService, OutputFormat, TableName, ColumnNames,
+    MaxValueColumnNames, WhereClause, MaxRowsPerFlowFile});
 
-    ok_ = true;
-  }
+  //! Set the supported relationships
+  setSupportedRelationships({ Success });
+}
 
-  explicit operator bool() const {
-    return ok_;
-  }
+void QueryDatabaseTable::processOnSchedule(core::ProcessContext& context) {
+  context.getProperty(OutputFormat.getName(), output_format_);
+  max_rows_ = [&] {
+    uint64_t max_rows;
+    context.getProperty(MaxRowsPerFlowFile.getName(), max_rows);
+    return gsl::narrow<size_t>(max_rows);
+  }();
 
-  const std::unordered_map<std::string, std::string>& getStateMap() const {
-    return mapState_;
+  state_manager_ = context.getStateManager();
+  if (state_manager_ == nullptr) {
+    throw Exception(PROCESSOR_EXCEPTION, "Failed to get StateManager");
   }
 
-  bool moveStateFileToMigrated() {
-    if (!ok_) {
-      return false;
-    }
-    return rename(filePath_.c_str(), (filePath_ + "-migrated").c_str()) == 0;
+  context.getProperty(TableName.getName(), table_name_);
+  context.getProperty(WhereClause.getName(), extra_where_clause_);
+  max_value_columns_ = [&] {
+    std::string max_value_columns_str;
+    context.getProperty(MaxValueColumnNames.getName(), max_value_columns_str);
+    return utils::inputStringToList(max_value_columns_str);
+  }();
+  return_columns_ = [&] {
+    std::string return_columns_str;
+    context.getProperty(ColumnNames.getName(), return_columns_str);
+    return utils::inputStringToList(return_columns_str);
+  }();
+  queried_columns_ = utils::StringUtils::join(", ", return_columns_);
+  if (!queried_columns_.empty() && !max_value_columns_.empty()) {
+    // columns will be explicitly enumerated, we need to add the max value columns
+    queried_columns_ = queried_columns_ + ", " + utils::StringUtils::join(", ", max_value_columns_);
   }
 
- private:
-  static const std::string separator_;
-
-   bool getStateFromFile() {
-     std::string state;
-
-     std::ifstream file(filePath_);
-     if (!file) {
-       return false;
-     }
-
-     std::stringstream ss;
-     ss << file.rdbuf();
-
-     state = ss.str();
-
-     file.close();
-
-     std::vector<std::string> listColumnNameValue;
-
-     size_t pos = state.find(separator_, 0);
-     if (pos == std::string::npos) {
-       logger_->log_error("Invalid data in '%s' file.", filePath_.c_str());
-       mapState_.clear();
-       return false;
-     }
-
-     auto tableName = state.substr(0, pos);
-     if (tableName != tableName_) {
-       logger_->log_warn("tableName is changed - now: '%s', in State.txt: '%s'.", tableName_.c_str(), tableName.c_str());
-       mapState_.clear();
-
-       return false;
-     }
-
-     pos += separator_.size();
-
-     while (true) {
-       auto newPos = state.find(separator_, pos);
-       if (newPos == std::string::npos)
-         break;
-
-       const std::string& columnNameValue = state.substr(pos, newPos - pos);
-       listColumnNameValue.emplace_back(columnNameValue);
-
-       pos = newPos + separator_.size();
-     }
-
-     for (const auto& columnNameValue : listColumnNameValue) {
-       const auto posEQ = columnNameValue.find('=');
-       if (posEQ == std::string::npos) {
-         logger_->log_error("Invalid data in '%s' file.", filePath_.c_str());
-         mapState_.clear();
-         return false;
-       }
-
-       const auto& name = columnNameValue.substr(0, posEQ);
-       const auto& value = columnNameValue.substr(posEQ + 1);
-
-       mapState_.insert({ name, value });
-     }
-
-     return true;
-   }
-
- private:
-   std::unordered_map<std::string, std::string> mapState_;
-   std::shared_ptr<logging::Logger> logger_;
-   std::string filePath_;
-   std::string tableName_;
-   bool ok_{};
-};
-
-const std::string LegacyState::separator_ = "@!qdt!@";
-
-// QueryDatabaseTable
-QueryDatabaseTable::QueryDatabaseTable(const std::string& name, utils::Identifier uuid)
-  : SQLProcessor(name, uuid) {
+  initializeMaxValues(context);
 }
 
-QueryDatabaseTable::~QueryDatabaseTable() = default;
-
-void QueryDatabaseTable::initialize() {
-  //! Set the supported properties
-  setSupportedProperties({ dbControllerService(), outputFormat(), s_tableName, s_columnNames, s_maxValueColumnNames, s_whereClause, s_sqlQuery, s_maxRowsPerFlowFile, s_stateDirectory});
+void QueryDatabaseTable::processOnTrigger(core::ProcessContext& context, core::ProcessSession& session) {
+  const auto& selectQuery = buildSelectQuery();
 
-  //! Set the supported relationships
-  setSupportedRelationships({ s_success });
-}
+  logger_->log_info("QueryDatabaseTable: selectQuery: '%s'", selectQuery.c_str());
 
-void QueryDatabaseTable::processOnSchedule(core::ProcessContext &context) {
-  initOutputFormat(context);
+  auto statement = connection_->prepareStatement(selectQuery);
 
-  context.getProperty(s_tableName.getName(), tableName_);
-  context.getProperty(s_columnNames.getName(), columnNames_);
+  auto rowset = statement->execute();
 
-  context.getProperty(s_maxValueColumnNames.getName(), maxValueColumnNames_);
-  listMaxValueColumnName_ = utils::inputStringToList(maxValueColumnNames_);
+  std::unordered_map<std::string, std::string> new_max_values = max_values_;
+  sql::MaxCollector maxCollector{selectQuery, new_max_values};
+  auto column_filter = [&] (const std::string& column_name) {
+    return return_columns_.empty()
+      || std::find(return_columns_.begin(), return_columns_.end(), column_name) != return_columns_.end();
+  };
+  sql::JSONSQLWriter sqlWriter{output_format_ == OutputType::JSONPretty, column_filter};
+  FlowFileGenerator flow_file_creator{session, sqlWriter};
+  sql::SQLRowsetProcessor sqlRowsetProcessor(rowset, {sqlWriter, maxCollector, flow_file_creator});
+
+  while (size_t row_count = sqlRowsetProcessor.process(max_rows_)) {
+    auto new_file = flow_file_creator.getLastFlowFile();
+    new_file->addAttribute(RESULT_ROW_COUNT, std::to_string(row_count));
+    new_file->addAttribute(RESULT_TABLE_NAME, table_name_);
+  }
 
-  context.getProperty(s_whereClause.getName(), whereClause_);
-  context.getProperty(s_sqlQuery.getName(), sqlQuery_);
-  context.getProperty(s_maxRowsPerFlowFile.getName(), maxRowsPerFlowFile_);
+  // the updated max_values and the total number of flow_files is available from here
+  for (auto& new_file : flow_file_creator.getFlowFiles()) {
+    session.transfer(new_file, Success);
+    for (const auto& max_column : max_value_columns_) {
+      new_file->addAttribute("maxvalue." + max_column, new_max_values[max_column]);
+    }
+  }
 
-  mapState_.clear();
+  if (new_max_values != max_values_) {
+    try {
+      session.commit();
+    } catch (std::exception& e) {
+      throw;
+    }

Review comment:
       done

##########
File path: extensions/sql/processors/QueryDatabaseTable.cpp
##########
@@ -75,361 +72,232 @@ const core::Property QueryDatabaseTable::s_maxValueColumnNames(
     "If no columns are provided, all rows from the table will be considered, which could have a performance impact. "
     "NOTE: It is important to use consistent max-value column names for a given table for incremental fetch to work properly. "
     "NOTE: Because of a limitation of database access library 'soci', which doesn't support milliseconds in it's 'dt_date', "
-    "there is a possibility that flowfiles might have duplicated records, if a max-value column with 'dt_date' type has value with milliseconds.")->
-    supportsExpressionLanguage(true)->build());
+    "there is a possibility that flowfiles might have duplicated records, if a max-value column with 'dt_date' type has value with milliseconds.")
+  ->supportsExpressionLanguage(true)->build());
 
-const core::Property QueryDatabaseTable::s_whereClause(
-  core::PropertyBuilder::createProperty("db-fetch-where-clause")->isRequired(false)->withDescription(
-    "A custom clause to be added in the WHERE condition when building SQL queries.")->supportsExpressionLanguage(true)->build());
+const core::Property QueryDatabaseTable::WhereClause(
+  core::PropertyBuilder::createProperty("Where Clause")
+  ->isRequired(false)
+  ->withDescription("A custom clause to be added in the WHERE condition when building SQL queries.")
+  ->supportsExpressionLanguage(true)->build());
 
-const core::Property QueryDatabaseTable::s_sqlQuery(
-  core::PropertyBuilder::createProperty("db-fetch-sql-query")->isRequired(false)->withDescription(
-    "A custom SQL query used to retrieve data. Instead of building a SQL query from other properties, this query will be wrapped as a sub-query. "
-    "Query must have no ORDER BY statement.")->supportsExpressionLanguage(true)->build());
+const std::string QueryDatabaseTable::InitialMaxValueDynamicPropertyPrefix("initial.maxvalue.");
 
-const core::Property QueryDatabaseTable::s_maxRowsPerFlowFile(
-  core::PropertyBuilder::createProperty("qdbt-max-rows")->isRequired(true)->withDefaultValue<int>(0)->withDescription(
-    "The maximum number of result rows that will be included in a single FlowFile. This will allow you to break up very large result sets into multiple FlowFiles. "
-    "If the value specified is zero, then all rows are returned in a single FlowFile.")->supportsExpressionLanguage(true)->build());
+const core::Relationship QueryDatabaseTable::Success("success", "Successfully created FlowFile from SQL query result set.");
 
-const core::Property QueryDatabaseTable::s_stateDirectory(
-  core::PropertyBuilder::createProperty("State Directory")->isRequired(false)->withDefaultValue("QDTState")->withDescription("DEPRECATED. Only use it for state migration from the state file, supplying the legacy state directory.")->build());
+const std::string QueryDatabaseTable::RESULT_TABLE_NAME = "tablename";
+const std::string QueryDatabaseTable::RESULT_ROW_COUNT = "querydbtable.row.count";
 
-const std::string QueryDatabaseTable::s_initialMaxValueDynamicPropertyPrefix("initial.maxvalue.");
+const std::string QueryDatabaseTable::TABLENAME_KEY = "tablename";
+const std::string QueryDatabaseTable::MAXVALUE_KEY_PREFIX = "maxvalue.";
 
-const core::Relationship QueryDatabaseTable::s_success("success", "Successfully created FlowFile from SQL query result set.");
-
-static const std::string ResultTableName = "tablename";
-static const std::string ResultRowCount = "querydbtable.row.count";
-
-static const std::string TABLENAME_KEY = "tablename";
-static const std::string MAXVALUE_KEY_PREFIX = "maxvalue.";
-
-// State
-class LegacyState {
- public:
-  LegacyState(const std::string& tableName, const std::string& stateDir, const std::string& uuid, std::shared_ptr<logging::Logger> logger)
-    :tableName_(tableName), logger_(logger) {
-
-    filePath_ = utils::file::FileUtils::concat_path(
-            utils::file::FileUtils::concat_path(
-              utils::file::FileUtils::concat_path(stateDir, "uuid"), uuid), "State.txt");
+// QueryDatabaseTable
+QueryDatabaseTable::QueryDatabaseTable(const std::string& name, utils::Identifier uuid)
+  : SQLProcessor(name, uuid, logging::LoggerFactory<QueryDatabaseTable>::getLogger()) {
+}
 
-    if (!getStateFromFile())
-      return;
+void QueryDatabaseTable::initialize() {
+  //! Set the supported properties
+  setSupportedProperties({
+    DBControllerService, OutputFormat, TableName, ColumnNames,
+    MaxValueColumnNames, WhereClause, MaxRowsPerFlowFile});
 
-    ok_ = true;
-  }
+  //! Set the supported relationships
+  setSupportedRelationships({ Success });
+}
 
-  explicit operator bool() const {
-    return ok_;
-  }
+void QueryDatabaseTable::processOnSchedule(core::ProcessContext& context) {
+  context.getProperty(OutputFormat.getName(), output_format_);
+  max_rows_ = [&] {
+    uint64_t max_rows;
+    context.getProperty(MaxRowsPerFlowFile.getName(), max_rows);
+    return gsl::narrow<size_t>(max_rows);
+  }();
 
-  const std::unordered_map<std::string, std::string>& getStateMap() const {
-    return mapState_;
+  state_manager_ = context.getStateManager();
+  if (state_manager_ == nullptr) {
+    throw Exception(PROCESSOR_EXCEPTION, "Failed to get StateManager");
   }
 
-  bool moveStateFileToMigrated() {
-    if (!ok_) {
-      return false;
-    }
-    return rename(filePath_.c_str(), (filePath_ + "-migrated").c_str()) == 0;
+  context.getProperty(TableName.getName(), table_name_);
+  context.getProperty(WhereClause.getName(), extra_where_clause_);
+  max_value_columns_ = [&] {
+    std::string max_value_columns_str;
+    context.getProperty(MaxValueColumnNames.getName(), max_value_columns_str);
+    return utils::inputStringToList(max_value_columns_str);
+  }();
+  return_columns_ = [&] {
+    std::string return_columns_str;
+    context.getProperty(ColumnNames.getName(), return_columns_str);
+    return utils::inputStringToList(return_columns_str);
+  }();
+  queried_columns_ = utils::StringUtils::join(", ", return_columns_);
+  if (!queried_columns_.empty() && !max_value_columns_.empty()) {
+    // columns will be explicitly enumerated, we need to add the max value columns
+    queried_columns_ = queried_columns_ + ", " + utils::StringUtils::join(", ", max_value_columns_);
   }
 
- private:
-  static const std::string separator_;
-
-   bool getStateFromFile() {
-     std::string state;
-
-     std::ifstream file(filePath_);
-     if (!file) {
-       return false;
-     }
-
-     std::stringstream ss;
-     ss << file.rdbuf();
-
-     state = ss.str();
-
-     file.close();
-
-     std::vector<std::string> listColumnNameValue;
-
-     size_t pos = state.find(separator_, 0);
-     if (pos == std::string::npos) {
-       logger_->log_error("Invalid data in '%s' file.", filePath_.c_str());
-       mapState_.clear();
-       return false;
-     }
-
-     auto tableName = state.substr(0, pos);
-     if (tableName != tableName_) {
-       logger_->log_warn("tableName is changed - now: '%s', in State.txt: '%s'.", tableName_.c_str(), tableName.c_str());
-       mapState_.clear();
-
-       return false;
-     }
-
-     pos += separator_.size();
-
-     while (true) {
-       auto newPos = state.find(separator_, pos);
-       if (newPos == std::string::npos)
-         break;
-
-       const std::string& columnNameValue = state.substr(pos, newPos - pos);
-       listColumnNameValue.emplace_back(columnNameValue);
-
-       pos = newPos + separator_.size();
-     }
-
-     for (const auto& columnNameValue : listColumnNameValue) {
-       const auto posEQ = columnNameValue.find('=');
-       if (posEQ == std::string::npos) {
-         logger_->log_error("Invalid data in '%s' file.", filePath_.c_str());
-         mapState_.clear();
-         return false;
-       }
-
-       const auto& name = columnNameValue.substr(0, posEQ);
-       const auto& value = columnNameValue.substr(posEQ + 1);
-
-       mapState_.insert({ name, value });
-     }
-
-     return true;
-   }
-
- private:
-   std::unordered_map<std::string, std::string> mapState_;
-   std::shared_ptr<logging::Logger> logger_;
-   std::string filePath_;
-   std::string tableName_;
-   bool ok_{};
-};
-
-const std::string LegacyState::separator_ = "@!qdt!@";
-
-// QueryDatabaseTable
-QueryDatabaseTable::QueryDatabaseTable(const std::string& name, utils::Identifier uuid)
-  : SQLProcessor(name, uuid) {
+  initializeMaxValues(context);
 }
 
-QueryDatabaseTable::~QueryDatabaseTable() = default;
-
-void QueryDatabaseTable::initialize() {
-  //! Set the supported properties
-  setSupportedProperties({ dbControllerService(), outputFormat(), s_tableName, s_columnNames, s_maxValueColumnNames, s_whereClause, s_sqlQuery, s_maxRowsPerFlowFile, s_stateDirectory});
+void QueryDatabaseTable::processOnTrigger(core::ProcessContext& context, core::ProcessSession& session) {
+  const auto& selectQuery = buildSelectQuery();
 
-  //! Set the supported relationships
-  setSupportedRelationships({ s_success });
-}
+  logger_->log_info("QueryDatabaseTable: selectQuery: '%s'", selectQuery.c_str());
 
-void QueryDatabaseTable::processOnSchedule(core::ProcessContext &context) {
-  initOutputFormat(context);
+  auto statement = connection_->prepareStatement(selectQuery);
 
-  context.getProperty(s_tableName.getName(), tableName_);
-  context.getProperty(s_columnNames.getName(), columnNames_);
+  auto rowset = statement->execute();
 
-  context.getProperty(s_maxValueColumnNames.getName(), maxValueColumnNames_);
-  listMaxValueColumnName_ = utils::inputStringToList(maxValueColumnNames_);
+  std::unordered_map<std::string, std::string> new_max_values = max_values_;
+  sql::MaxCollector maxCollector{selectQuery, new_max_values};
+  auto column_filter = [&] (const std::string& column_name) {
+    return return_columns_.empty()
+      || std::find(return_columns_.begin(), return_columns_.end(), column_name) != return_columns_.end();
+  };
+  sql::JSONSQLWriter sqlWriter{output_format_ == OutputType::JSONPretty, column_filter};
+  FlowFileGenerator flow_file_creator{session, sqlWriter};
+  sql::SQLRowsetProcessor sqlRowsetProcessor(rowset, {sqlWriter, maxCollector, flow_file_creator});
+
+  while (size_t row_count = sqlRowsetProcessor.process(max_rows_)) {
+    auto new_file = flow_file_creator.getLastFlowFile();
+    new_file->addAttribute(RESULT_ROW_COUNT, std::to_string(row_count));
+    new_file->addAttribute(RESULT_TABLE_NAME, table_name_);
+  }
 
-  context.getProperty(s_whereClause.getName(), whereClause_);
-  context.getProperty(s_sqlQuery.getName(), sqlQuery_);
-  context.getProperty(s_maxRowsPerFlowFile.getName(), maxRowsPerFlowFile_);
+  // the updated max_values and the total number of flow_files is available from here
+  for (auto& new_file : flow_file_creator.getFlowFiles()) {
+    session.transfer(new_file, Success);
+    for (const auto& max_column : max_value_columns_) {
+      new_file->addAttribute("maxvalue." + max_column, new_max_values[max_column]);
+    }
+  }
 
-  mapState_.clear();
+  if (new_max_values != max_values_) {
+    try {
+      session.commit();
+    } catch (std::exception& e) {
+      throw;
+    }
 
-  state_manager_ = context.getStateManager();
-  if (state_manager_ == nullptr) {
-    throw Exception(PROCESSOR_EXCEPTION, "Failed to get StateManager");
+    max_values_ = new_max_values;
+    saveState();
   }
+}
 
-  std::unordered_map<std::string, std::string> state_map;
-  if (state_manager_->get(state_map)) {
-    if (state_map[TABLENAME_KEY] != tableName_) {
-      state_manager_->clear();
-    } else {
-      for (auto&& elem : state_map) {
-        if (elem.first.find(MAXVALUE_KEY_PREFIX) == 0) {
-          mapState_.emplace(elem.first.substr(MAXVALUE_KEY_PREFIX.length()), std::move(elem.second));
+void QueryDatabaseTable::initializeMaxValues(core::ProcessContext &context) {
+  max_values_.clear();
+  std::unordered_map<std::string, std::string> new_state;
+  if (!state_manager_->get(new_state)) {
+    logger_->log_info("Found no stored state");
+  } else {
+    const bool should_reset_state = [&] {

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