You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "martinzink (via GitHub)" <gi...@apache.org> on 2023/06/30 13:44:52 UTC

[GitHub] [nifi-minifi-cpp] martinzink commented on a diff in pull request #1589: MINIFICPP-1825 Create Properties at compile time

martinzink commented on code in PR #1589:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1589#discussion_r1247874034


##########
extensions/aws/processors/S3Processor.h:
##########
@@ -83,23 +90,67 @@ struct CommonProperties {
 
 class S3Processor : public core::Processor {
  public:
-  static const std::set<std::string> REGIONS;
-
-  EXTENSIONAPI static const core::Property Bucket;
-  EXTENSIONAPI static const core::Property AccessKey;
-  EXTENSIONAPI static const core::Property SecretKey;
-  EXTENSIONAPI static const core::Property CredentialsFile;
-  EXTENSIONAPI static const core::Property AWSCredentialsProviderService;
-  EXTENSIONAPI static const core::Property Region;
-  EXTENSIONAPI static const core::Property CommunicationsTimeout;
-  EXTENSIONAPI static const core::Property EndpointOverrideURL;
-  EXTENSIONAPI static const core::Property ProxyHost;
-  EXTENSIONAPI static const core::Property ProxyPort;
-  EXTENSIONAPI static const core::Property ProxyUsername;
-  EXTENSIONAPI static const core::Property ProxyPassword;
-  EXTENSIONAPI static const core::Property UseDefaultCredentials;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto Bucket = core::PropertyDefinitionBuilder<>::createProperty("Bucket")
+      .withDescription("The S3 bucket")
+      .isRequired(true)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto AccessKey = core::PropertyDefinitionBuilder<>::createProperty("Access Key")
+      .withDescription("AWS account access key")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto SecretKey = core::PropertyDefinitionBuilder<>::createProperty("Secret Key")
+      .withDescription("AWS account secret key")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto CredentialsFile = core::PropertyDefinitionBuilder<>::createProperty("Credentials File")
+      .withDescription("Path to a file containing AWS access key and secret key in properties file format. Properties used: accessKey and secretKey")
+      .build();
+  EXTENSIONAPI static constexpr auto AWSCredentialsProviderService = core::PropertyDefinitionBuilder<>::createProperty("AWS Credentials Provider service")
+      .withDescription("The name of the AWS Credentials Provider controller service that is used to obtain AWS credentials.")
+      .build();
+  EXTENSIONAPI static constexpr auto Region = core::PropertyDefinitionBuilder<region::REGIONS.size()>::createProperty("Region")
+      .isRequired(true)
+      .withDefaultValue(region::US_WEST_2)
+      .withAllowedValues(region::REGIONS)
+      .withDescription("AWS Region")
+      .build();
+  EXTENSIONAPI static constexpr auto CommunicationsTimeout = core::PropertyDefinitionBuilder<>::createProperty("Communications Timeout")
+      .isRequired(true)
+      .withPropertyType(core::StandardPropertyTypes::TIME_PERIOD_TYPE)
+      .withDefaultValue("30 sec")
+      .withDescription("Sets the timeout of the communication between the AWS server and the client")
+      .build();
+  EXTENSIONAPI static constexpr auto EndpointOverrideURL = core::PropertyDefinitionBuilder<>::createProperty("Endpoint Override URL")
+      .withDescription("Endpoint URL to use instead of the AWS default including scheme, host, "
+          "port, and path. The AWS libraries select an endpoint URL based on the AWS "
+          "region, but this property overrides the selected endpoint URL, allowing use "
+          "with other S3-compatible endpoints.")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ProxyHost = core::PropertyDefinitionBuilder<>::createProperty("Proxy Host")
+      .withDescription("Proxy host name or IP")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ProxyPort = core::PropertyDefinitionBuilder<>::createProperty("Proxy Port")
+      .withDescription("The port number of the proxy host")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ProxyUsername =   core::PropertyDefinitionBuilder<>::createProperty("Proxy Username")

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto ProxyUsername = core::PropertyDefinitionBuilder<>::createProperty("Proxy Username")
   ```



##########
extensions/gcp/processors/ListGCSBucket.h:
##########
@@ -37,66 +51,71 @@ class ListGCSBucket : public GCSProcessor {
   EXTENSIONAPI static constexpr const char* Description = "Retrieves a listing of objects from an GCS bucket. "
       "For each object that is listed, creates a FlowFile that represents the object so that it can be fetched in conjunction with FetchGCSObject.";
 
-  EXTENSIONAPI static const core::Property Bucket;
-  EXTENSIONAPI static const core::Property ListAllVersions;
-  static auto properties() {
-    return utils::array_cat(GCSProcessor::properties(), std::array{
+  EXTENSIONAPI static constexpr auto Bucket = core::PropertyDefinitionBuilder<>::createProperty("Bucket")
+      .withDescription("Bucket of the object.")
+      .isRequired(true)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ListAllVersions = core::PropertyDefinitionBuilder<>::createProperty("List all versions")
+      .withDescription("Set this option to `true` to get all the previous versions separately.")
+      .withPropertyType(core::StandardPropertyTypes::BOOLEAN_TYPE)
+      .withDefaultValue("false")
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = utils::array_cat(GCSProcessor::Properties, std::array<core::PropertyReference, 2>{
       Bucket,
       ListAllVersions
-    });
-  }
+  });
 
-  EXTENSIONAPI static const core::Relationship Success;
-  static auto relationships() { return std::array{Success}; }
-
-  EXTENSIONAPI static const core::OutputAttribute BucketOutputAttribute;
-  EXTENSIONAPI static const core::OutputAttribute Key;
-  EXTENSIONAPI static const core::OutputAttribute Filename;
-  EXTENSIONAPI static const core::OutputAttribute Size;
-  EXTENSIONAPI static const core::OutputAttribute Crc32c;
-  EXTENSIONAPI static const core::OutputAttribute Md5;
-  EXTENSIONAPI static const core::OutputAttribute OwnerEntity;
-  EXTENSIONAPI static const core::OutputAttribute OwnerEntityId;
-  EXTENSIONAPI static const core::OutputAttribute ContentEncoding;
-  EXTENSIONAPI static const core::OutputAttribute ContentLanguage;
-  EXTENSIONAPI static const core::OutputAttribute ContentDisposition;
-  EXTENSIONAPI static const core::OutputAttribute MediaLink;
-  EXTENSIONAPI static const core::OutputAttribute SelfLink;
-  EXTENSIONAPI static const core::OutputAttribute Etag;
-  EXTENSIONAPI static const core::OutputAttribute GeneratedId;
-  EXTENSIONAPI static const core::OutputAttribute Generation;
-  EXTENSIONAPI static const core::OutputAttribute Metageneration;
-  EXTENSIONAPI static const core::OutputAttribute CreateTime;
-  EXTENSIONAPI static const core::OutputAttribute UpdateTime;
-  EXTENSIONAPI static const core::OutputAttribute DeleteTime;
-  EXTENSIONAPI static const core::OutputAttribute EncryptionAlgorithm;
-  EXTENSIONAPI static const core::OutputAttribute EncryptionSha256;
-  static auto outputAttributes() {
-    return std::array{
-        BucketOutputAttribute,
-        Key,
-        Filename,
-        Size,
-        Crc32c,
-        Md5,
-        OwnerEntity,
-        OwnerEntityId,
-        ContentEncoding,
-        ContentLanguage,
-        ContentDisposition,
-        MediaLink,
-        SelfLink,
-        Etag,
-        GeneratedId,
-        Generation,
-        Metageneration,
-        CreateTime,
-        UpdateTime,
-        DeleteTime,
-        EncryptionAlgorithm,
-        EncryptionSha256
-    };
-  }
+
+  EXTENSIONAPI static constexpr auto Success =  core::RelationshipDefinition{"success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation."};

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Success = core::RelationshipDefinition{"success", "FlowFiles are routed to this relationship after a successful Google Cloud Storage operation."};
   ```



##########
extensions/pcap/CapturePacket.h:
##########
@@ -99,21 +103,34 @@ class CapturePacket : public core::Processor {
     " Configuration options exist to adjust the batching of PCAP files. PCAP batching will place a single PCAP into a flow file. "
     "A regular expression selects network interfaces. Bluetooth network interfaces can be selected through a separate option.";
 
-  static const core::Property BatchSize;
-  static const core::Property NetworkControllers;
-  static const core::Property BaseDir;
-  static const core::Property CaptureBluetooth;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto BatchSize = core::PropertyDefinitionBuilder<>::createProperty("Batch Size")
+      .withDescription("The number of packets to combine within a given PCAP")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("50")
+      .build();
+  EXTENSIONAPI static constexpr auto NetworkControllers = core::PropertyDefinitionBuilder<>::createProperty("Network Controllers")
+      .withDescription("Regular expression of the network controller(s) to which we will attach")
+      .withDefaultValue(".*")
+      .build();
+  EXTENSIONAPI static constexpr auto BaseDir = core::PropertyDefinitionBuilder<>::createProperty("Base Directory")
+      .withDescription("Scratch directory for PCAP files")
+      .withDefaultValue("/tmp/")
+      .build();
+  EXTENSIONAPI static constexpr auto CaptureBluetooth = core::PropertyDefinitionBuilder<>::createProperty("Capture Bluetooth")
+      .withDescription("True indicates that we support bluetooth interfaces")
+      .withPropertyType(core::StandardPropertyTypes::BOOLEAN_TYPE)
+      .withDefaultValue("false")
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = std::array<core::PropertyReference, 4>{
       BatchSize,
       NetworkControllers,
       BaseDir,
       CaptureBluetooth
-    };
-  }
+  };
+
 
-  EXTENSIONAPI static const core::Relationship Success;
-  static auto relationships() { return std::array{Success}; }
+  EXTENSIONAPI static constexpr auto Success =  core::RelationshipDefinition{"success", "All files are routed to success"};

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Success = core::RelationshipDefinition{"success", "All files are routed to success"};
   ```



##########
extensions/sql/processors/ExecuteSQL.h:
##########
@@ -41,14 +43,19 @@ class ExecuteSQL : public SQLProcessor, public FlowFileSource {
       "This processor can be scheduled to run using the standard timer-based scheduling methods, or it can be triggered by an incoming FlowFile. "
       "If it is triggered by an incoming FlowFile, then attributes of that FlowFile will be available when evaluating the query.";
 
-  EXTENSIONAPI static const core::Property SQLSelectQuery;
-  static auto properties() {
-    return utils::array_cat(SQLProcessor::properties(), FlowFileSource::properties(), std::array{SQLSelectQuery});
-  }
+  EXTENSIONAPI static constexpr auto SQLSelectQuery = core::PropertyDefinitionBuilder<>::createProperty("SQL select query")
+      .withDescription(
+        "The SQL select query to execute. The query can be empty, a constant value, or built from attributes using Expression Language. "
+        "If this property is specified, it will be used regardless of the content of incoming flowfiles. "
+        "If this property is empty, the content of the incoming flow file is expected to contain a valid SQL select query, to be issued by the processor to the database. "
+        "Note that Expression Language is not evaluated for flow file contents.")
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = utils::array_cat(SQLProcessor::Properties, FlowFileSource::Properties, std::array<core::PropertyReference, 1>{SQLSelectQuery});
 
-  EXTENSIONAPI static const core::Relationship Success;
-  EXTENSIONAPI static const core::Relationship Failure;
-  static auto relationships() { return std::array{Success, Failure}; }
+  EXTENSIONAPI static constexpr auto Success =  core::RelationshipDefinition{"success", "Successfully created FlowFile from SQL query result set."};
+  EXTENSIONAPI static constexpr auto Failure =  core::RelationshipDefinition{"failure", "Flow files containing malformed sql statements"};

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Success = core::RelationshipDefinition{"success", "Successfully created FlowFile from SQL query result set."};
     EXTENSIONAPI static constexpr auto Failure = core::RelationshipDefinition{"failure", "Flow files containing malformed sql statements"};
   ```



##########
extensions/standard-processors/tests/unit/ProcessorTests.cpp:
##########
@@ -496,9 +494,9 @@ class TestProcessorNoContent : public minifi::core::Processor {
   }
 
   static constexpr const char* Description = "test resource";
-  static auto properties() { return std::array<core::Property, 0>{}; }
-  static const core::Relationship Success;
-  static auto relationships() { return std::array{Success}; }
+  static constexpr auto Properties = std::array<core::PropertyReference, 0>{};
+  static constexpr auto Success =  core::RelationshipDefinition{"success", "All files are routed to success"};

Review Comment:
   ```suggestion
     static constexpr auto Success = core::RelationshipDefinition{"success", "All files are routed to success"};
   ```



##########
extensions/standard-processors/processors/GenerateFlowFile.h:
##########
@@ -47,23 +51,45 @@ class GenerateFlowFile : public core::Processor {
   EXTENSIONAPI static constexpr const char* Description = "This processor creates FlowFiles with random data or custom content. "
       "GenerateFlowFile is useful for load testing, configuration, and simulation.";
 
-  EXTENSIONAPI static const core::Property FileSize;
-  EXTENSIONAPI static const core::Property BatchSize;
-  EXTENSIONAPI static const core::Property DataFormat;
-  EXTENSIONAPI static const core::Property UniqueFlowFiles;
-  EXTENSIONAPI static const core::Property CustomText;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto FileSize = core::PropertyDefinitionBuilder<>::createProperty("File Size")
+      .withDescription("The size of the file that will be used")
+      .isRequired(false)
+      .withPropertyType(core::StandardPropertyTypes::DATA_SIZE_TYPE)
+      .withDefaultValue("1 kB")
+      .build();
+  EXTENSIONAPI static constexpr auto BatchSize = core::PropertyDefinitionBuilder<>::createProperty("Batch Size")
+      .withDescription("The number of FlowFiles to be transferred in each invocation")
+      .isRequired(false)
+      .withPropertyType(core::StandardPropertyTypes::INTEGER_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto DataFormat =  core::PropertyDefinitionBuilder<2>::createProperty("Data Format")

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto DataFormat = core::PropertyDefinitionBuilder<2>::createProperty("Data Format")
   ```



##########
extensions/libarchive/BinFiles.h:
##########
@@ -199,33 +202,57 @@ class BinFiles : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* Description = "Bins flow files into buckets based on the number of entries or size of entries";
 
-  EXTENSIONAPI static const core::Property MinSize;
-  EXTENSIONAPI static const core::Property MaxSize;
-  EXTENSIONAPI static const core::Property MinEntries;
-  EXTENSIONAPI static const core::Property MaxEntries;
-  EXTENSIONAPI static const core::Property MaxBinCount;
-  EXTENSIONAPI static const core::Property MaxBinAge;
-  EXTENSIONAPI static const core::Property BatchSize;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto MinSize = core::PropertyDefinitionBuilder<>::createProperty("Minimum Group Size")
+      .withDescription("The minimum size of for the bundle")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_LONG_TYPE)
+      .withDefaultValue("0")
+      .build();
+  EXTENSIONAPI static constexpr auto MaxSize = core::PropertyDefinitionBuilder<>::createProperty("Maximum Group Size")
+      .withDescription("The maximum size for the bundle. If not specified, there is no maximum.")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_LONG_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MinEntries = core::PropertyDefinitionBuilder<>::createProperty("Minimum Number of Entries")
+      .withDescription("The minimum number of files to include in a bundle")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto MaxEntries = core::PropertyDefinitionBuilder<>::createProperty("Maximum Number of Entries")
+      .withDescription("The maximum number of files to include in a bundle. If not specified, there is no maximum.")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxBinAge = core::PropertyDefinitionBuilder<>::createProperty("Max Bin Age")
+      .withDescription("The maximum age of a Bin that will trigger a Bin to be complete. Expected format is <duration> <time unit>")
+      .withPropertyType(core::StandardPropertyTypes::TIME_PERIOD_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxBinCount = core::PropertyDefinitionBuilder<>::createProperty("Maximum number of Bins")
+      .withDescription("Specifies the maximum number of bins that can be held in memory at any one time")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("100")
+      .build();
+  EXTENSIONAPI static constexpr auto BatchSize = core::PropertyDefinitionBuilder<>::createProperty("Batch Size")
+      .withDescription("Maximum number of FlowFiles processed in a single session")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = std::array<core::PropertyReference, 7>{
       MinSize,
       MaxSize,
       MinEntries,
       MaxEntries,
       MaxBinCount,
       MaxBinAge,
       BatchSize
-    };
-  }
+  };
+
 
-  EXTENSIONAPI static const core::Relationship Failure;
-  EXTENSIONAPI static const core::Relationship Original;
-  static auto relationships() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto Failure =  core::RelationshipDefinition{"failure",

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Failure = core::RelationshipDefinition{"failure",
   ```



##########
extensions/libarchive/BinFiles.h:
##########
@@ -199,33 +202,57 @@ class BinFiles : public core::Processor {
 
   EXTENSIONAPI static constexpr const char* Description = "Bins flow files into buckets based on the number of entries or size of entries";
 
-  EXTENSIONAPI static const core::Property MinSize;
-  EXTENSIONAPI static const core::Property MaxSize;
-  EXTENSIONAPI static const core::Property MinEntries;
-  EXTENSIONAPI static const core::Property MaxEntries;
-  EXTENSIONAPI static const core::Property MaxBinCount;
-  EXTENSIONAPI static const core::Property MaxBinAge;
-  EXTENSIONAPI static const core::Property BatchSize;
-  static auto properties() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto MinSize = core::PropertyDefinitionBuilder<>::createProperty("Minimum Group Size")
+      .withDescription("The minimum size of for the bundle")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_LONG_TYPE)
+      .withDefaultValue("0")
+      .build();
+  EXTENSIONAPI static constexpr auto MaxSize = core::PropertyDefinitionBuilder<>::createProperty("Maximum Group Size")
+      .withDescription("The maximum size for the bundle. If not specified, there is no maximum.")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_LONG_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MinEntries = core::PropertyDefinitionBuilder<>::createProperty("Minimum Number of Entries")
+      .withDescription("The minimum number of files to include in a bundle")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto MaxEntries = core::PropertyDefinitionBuilder<>::createProperty("Maximum Number of Entries")
+      .withDescription("The maximum number of files to include in a bundle. If not specified, there is no maximum.")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxBinAge = core::PropertyDefinitionBuilder<>::createProperty("Max Bin Age")
+      .withDescription("The maximum age of a Bin that will trigger a Bin to be complete. Expected format is <duration> <time unit>")
+      .withPropertyType(core::StandardPropertyTypes::TIME_PERIOD_TYPE)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxBinCount = core::PropertyDefinitionBuilder<>::createProperty("Maximum number of Bins")
+      .withDescription("Specifies the maximum number of bins that can be held in memory at any one time")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("100")
+      .build();
+  EXTENSIONAPI static constexpr auto BatchSize = core::PropertyDefinitionBuilder<>::createProperty("Batch Size")
+      .withDescription("Maximum number of FlowFiles processed in a single session")
+      .withPropertyType(core::StandardPropertyTypes::UNSIGNED_INT_TYPE)
+      .withDefaultValue("1")
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = std::array<core::PropertyReference, 7>{
       MinSize,
       MaxSize,
       MinEntries,
       MaxEntries,
       MaxBinCount,
       MaxBinAge,
       BatchSize
-    };
-  }
+  };
+
 
-  EXTENSIONAPI static const core::Relationship Failure;
-  EXTENSIONAPI static const core::Relationship Original;
-  static auto relationships() {
-    return std::array{
+  EXTENSIONAPI static constexpr auto Failure =  core::RelationshipDefinition{"failure",
+      "If the bundle cannot be created, all FlowFiles that would have been used to create the bundle will be transferred to failure"};
+  EXTENSIONAPI static constexpr auto Original =  core::RelationshipDefinition{"original",

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Original = core::RelationshipDefinition{"original",
   ```



##########
extensions/sql/processors/QueryDatabaseTable.h:
##########
@@ -48,25 +50,56 @@ class QueryDatabaseTable: public SQLProcessor, public FlowFileSource {
       "Fetches all rows of a table, whose values in the specified Maximum-value Columns are larger than the previously-seen maxima. "
       "If that property is not provided, all rows are returned. The rows are grouped according to the value of Max Rows Per Flow File property and formatted as JSON.";
 
-  EXTENSIONAPI static const core::Property TableName;
-  EXTENSIONAPI static const core::Property ColumnNames;
-  EXTENSIONAPI static const core::Property MaxValueColumnNames;
-  EXTENSIONAPI static const core::Property WhereClause;
-  static auto properties() {
-    return utils::array_cat(SQLProcessor::properties(), FlowFileSource::properties(), std::array{
+  EXTENSIONAPI static constexpr auto TableName = core::PropertyDefinitionBuilder<>::createProperty("Table Name")
+      .withDescription("The name of the database table to be queried.")
+      .isRequired(true)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto ColumnNames = core::PropertyDefinitionBuilder<>::createProperty("Columns to Return")
+      .withDescription(
+        "A comma-separated list of column names to be used in the query. If your database requires special treatment of the names (quoting, e.g.), each name should include such treatment. "
+        "If no column names are supplied, all columns in the specified table will be returned. "
+        "NOTE: It is important to use consistent column names for a given table for incremental fetch to work properly.")
+      .isRequired(false)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto MaxValueColumnNames = core::PropertyDefinitionBuilder<>::createProperty("Maximum-value Columns")
+      .withDescription(
+        "A comma-separated list of column names. The processor will keep track of the maximum value for each column that has been returned since the processor started running. "
+        "Using multiple columns implies an order to the column list, and each column's values are expected to increase more slowly than the previous columns' values. "
+        "Thus, using multiple columns implies a hierarchical structure of columns, which is usually used for partitioning tables. "
+        "This processor can be used to retrieve only those rows that have been added/updated since the last retrieval. "
+        "Note that some ODBC types such as bit/boolean are not conducive to maintaining maximum value, so columns of these types should not be listed in this property, and will result in error(s) during processing. "
+        "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.")
+      .isRequired(false)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto WhereClause = core::PropertyDefinitionBuilder<>::createProperty("Where Clause")
+      .withDescription("A custom clause to be added in the WHERE condition when building SQL queries.")
+      .isRequired(false)
+      .supportsExpressionLanguage(true)
+      .build();
+  EXTENSIONAPI static constexpr auto Properties = utils::array_cat(SQLProcessor::Properties, FlowFileSource::Properties, std::array<core::PropertyReference, 4>{
       TableName,
       ColumnNames,
       MaxValueColumnNames,
       WhereClause
-    });
-  }
+  });
 
-  EXTENSIONAPI static const core::Relationship Success;
-  static auto relationships() { return std::array{Success}; }
+
+  EXTENSIONAPI static constexpr auto Success =  core::RelationshipDefinition{"success", "Successfully created FlowFile from SQL query result set."};

Review Comment:
   ```suggestion
     EXTENSIONAPI static constexpr auto Success = core::RelationshipDefinition{"success", "Successfully created FlowFile from SQL query result set."};
   ```



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org