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 2022/01/24 20:30:31 UTC

[GitHub] [nifi] davyam opened a new pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

davyam opened a new pull request #5710:
URL: https://github.com/apache/nifi/pull/5710


   <!--
     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.
   -->
   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   The CaptureChangePostgreSQL processor retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. This processor uses a Logical Replication Connection to stream data and a SQL Connection to query system views.
   
   This new pull request builds upon the original PR #4065, but all the code was refactored. Some improvements are:
   
   - Java best practices followed;
   - Optimized connection management;
   - Added cdc.lsn and cdc.type attributes;
   - Simplified change reading and decoding process;
   - Problematic Initial Snapshot functionality removed;
   - Better exception handling;
   - Javadoc descriptions improved;
   - Unit and Integration Tests created.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [X] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [X] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [X] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [X] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [X] Have you written or updated unit tests to verify your changes?
   - [X] Have you verified that the full build is successful on JDK 8?
   - [X] Have you verified that the full build is successful on JDK 11?
   - [X] 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)?
   - [X] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [X] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [X] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [X] 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 GitHub Actions 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.

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

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



[GitHub] [nifi] davyam edited a comment on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam edited a comment on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1024517839


   Hi @gerdansantos! As you suggested, the replication feedback now occurs after the flowfiles transfer to success relation completes. Thanks man!


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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795912483



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";
+        private static final String DRIVER_NAME = "org.postgresql.Driver";
+        private static final String HOST = "192.168.56.101";
+        private static final String PORT = "5432";
+        private static final String DATABASE = "db_test";
+        private static final String USERNAME = "nifi";
+        private static final String PASSWORD = "Change1t!";
+        private static final String CONNECTION_TIMEOUT = "30 seconds";
+        private static final String PUBLICATION = "pub_city";
+        private static final String REPLICATION_SLOT = "slot_city";
+
+        ITPostgreSQLClient client;
+        TestRunner runner;
+
+        @Before
+        public void setUp() throws Exception {
+                runner = TestRunners.newTestRunner(new CaptureChangePostgreSQL());
+                client = new ITPostgreSQLClient(HOST, PORT, DATABASE, USERNAME, PASSWORD);
+        }
+
+        @After
+        public void tearDown() throws Exception {
+                runner = null;
+
+                if (client != null) {
+                        client.closeConnection();
+                        client = null;
+                }
+        }
+
+        @Test
+        public void test01RequiredProperties() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+                runner.assertValid();
+        }
+
+        @Test
+        public void test02RegisterDriverFailure() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+                runner.assertValid();
+
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, "/tmp/postgrez.jar"); // Incorrect driver
+                                                                                                  // location
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.assertValid();
+
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, "org.postgrez.Driver"); // Incorrect driver name
+                try {
+                        runner.run();
+                } catch (AssertionError ae) {
+                        Throwable pe = ae.getCause();
+                        assertTrue(pe instanceof ProcessException);
+                        Throwable rue = pe.getCause();
+                        assertTrue(rue instanceof RuntimeException);
+                        assertEquals(
+                                        "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                                                        + "are configured correctly. org.apache.nifi.reporting.InitializationException: Can't load Database Driver",
+                                        rue.getMessage());
+                        Throwable ine = rue.getCause();
+                        assertTrue(ine instanceof InitializationException);
+                        assertEquals("Can't load Database Driver", ine.getMessage());
+                }
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+        }
+
+        @Test
+        public void test03ConnectionFailures() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+
+                runner.setProperty(CaptureChangePostgreSQL.HOST, "192.168.56.199"); // Incorrect IP
+                try {
+                        runner.run();
+                } catch (AssertionError ae) {
+                        Throwable pe = ae.getCause();
+                        assertTrue(pe instanceof ProcessException);
+                        Throwable ioe = pe.getCause();
+                        assertTrue(ioe instanceof IOException);
+                        assertEquals("Error creating SQL connection to specified host and port", ioe.getMessage());
+                        Throwable psqle = ioe.getCause();
+                        assertTrue(psqle instanceof PSQLException);
+                }

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.

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

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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795913982



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);
+        }
+    }
+
+    @OnStopped
+    public void onStopped(ProcessContext context) {
+        try {
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    @OnShutdown
+    public void onShutdown(ProcessContext context) {
+        try {
+            // In case we get shutdown while still running, save off the current state,
+            // disconnect, and shut down gracefully.
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    protected void createReplicationReader(String slot, boolean dropSlotIfExists, String publication, Long lsn,
+            boolean includeBeginCommit, boolean includeAllMetadata, Connection replicationConn,
+            Connection queryConn) throws SQLException {
+        this.replicationReader = new Reader(slot, dropSlotIfExists, publication, lsn, includeBeginCommit,
+                includeAllMetadata, replicationConn, queryConn);
+    }
+
+    protected Long getMaxFlowFileListSize() {
+        return this.maxFlowFileListSize;
+    }
+
+    protected void connect(String host, String port, String database, String username, String password,
+            String driverLocation, String driverName, long connectionTimeout)
+            throws IOException, TimeoutException {
+        try {
+            // Ensure driverLocation and driverName are correct
+            // before establishing connection.
+            registerDriver(driverLocation, driverName);
+        } catch (InitializationException e) {
+            throw new RuntimeException(
+                    "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                            + "are configured correctly. " + e,
+                    e);
+        }
+
+        // Connection expects a non-null password.
+        if (password == null) {
+            password = "";
+        }
+
+        // Connection expects a timeout.
+        if (connectionTimeout == 0) {
+            connectionTimeout = Long.MAX_VALUE;
+        }
+
+        InetSocketAddress address = getAddress(host, port);
+
+        queryConnHolder = new JDBCConnectionHolder(address, database, username, password, false, connectionTimeout);
+        try {
+            // Ensure Query connection can be created.
+            this.getQueryConnection();
+        } catch (SQLException e) {
+            throw new IOException("Error creating SQL connection to specified host and port", e);
+        }
+
+        replicationConnHolder = new JDBCConnectionHolder(address, database, username, password, true,
+                connectionTimeout);
+        try {
+            // Ensure Replication connection can be created.
+            this.getReplicationConnection();
+        } catch (SQLException e) {
+            throw new IOException("Error creating Replication connection to specified host and port", e);
+        }
+    }
+
+    protected Connection getQueryConnection() throws SQLException {
+        return queryConnHolder.getConnection();
+    }
+
+    protected Connection getReplicationConnection() throws SQLException {
+        return replicationConnHolder.getConnection();
+    }
+
+    protected void closeQueryConnection() {
+        if (queryConnHolder != null)
+            queryConnHolder.close();
+    }
+
+    protected void closeReplicationConnection() {
+        if (replicationConnHolder != null)
+            replicationConnHolder.close();

Review comment:
       Done! Suggestion included in last commit. Thanks!




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



[GitHub] [nifi] davyam edited a comment on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam edited a comment on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020553625


   Hey @rrjanbiah @driesva @mathiasbosman, 
   
   We believe that the improvements asked in the last PR #4065 were made. Feel free to check and test. We are counting on your feedback too.


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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795073671



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);

Review comment:
       Messages should not include exclamation marks, please update the message:
   ```suggestion
               throw new CDCException("Closing CDC Connection failed", e);
   ```




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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795089266



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);

Review comment:
       Sure! Thanks for the commit suggestion!




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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795913732



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);
+        }
+    }
+
+    @OnStopped
+    public void onStopped(ProcessContext context) {
+        try {
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    @OnShutdown
+    public void onShutdown(ProcessContext context) {
+        try {
+            // In case we get shutdown while still running, save off the current state,
+            // disconnect, and shut down gracefully.
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    protected void createReplicationReader(String slot, boolean dropSlotIfExists, String publication, Long lsn,
+            boolean includeBeginCommit, boolean includeAllMetadata, Connection replicationConn,
+            Connection queryConn) throws SQLException {
+        this.replicationReader = new Reader(slot, dropSlotIfExists, publication, lsn, includeBeginCommit,
+                includeAllMetadata, replicationConn, queryConn);
+    }
+
+    protected Long getMaxFlowFileListSize() {
+        return this.maxFlowFileListSize;
+    }
+
+    protected void connect(String host, String port, String database, String username, String password,
+            String driverLocation, String driverName, long connectionTimeout)
+            throws IOException, TimeoutException {
+        try {
+            // Ensure driverLocation and driverName are correct
+            // before establishing connection.
+            registerDriver(driverLocation, driverName);
+        } catch (InitializationException e) {
+            throw new RuntimeException(
+                    "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                            + "are configured correctly. " + e,

Review comment:
       Done! Suggestion included in last commit. Thanks!




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



[GitHub] [nifi] davyam edited a comment on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam edited a comment on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020720083


   > I have tested this code.
   > 
   > ✅ Build without exceptions ✅ The mock tests seem enough ✅ The CDC start without problems including PostgreSQL slots ✅ The Syncing Turn On process runs fine ❌ The recovery process, dont conect and recovery from abrutal shutdown of PostgreSQL of NiFi. I beleive the processor stat came be definied before PostgreSQL lsn Commit, so the PostgreSQL cannot resync with the last past LSN. ✅If the stop process, the processor restart from last step without problems. ❌ The connection with a slot previously created don't run.
   > 
   > Congrats for the evolution of this processor code.
   
   Hi man! Thanks for the feedback!
   
   About the recovery test, please give me more information about your simulation. The PostgreSQL Replication Slot doesn't keep changes after the feedback (setAppliedLSN and setFlushedLSN functions), which is performed as soon as the change is received by the processor. Then, if you try to set the Start LSN property with an LSN already received, that will not work and the processor will start from the pending changes available in the slot. But you can test the opposite, I mean, you can request an LSN after many unreceived transactions, so the processor will skip them and start from the informed LSN (since the state is clean). Take a look at the integration test testCDCStartLSN.
   
   Talking about the test with an existing slot, this is working. I tested it many times, including integration tests. I believe that, previously, you don't create the slot with **pgoutput** output plugin. For example:
   
   `SELECT pg_create_logical_replication_slot('slot_city', 'pgoutput');`
   
   Probably this is the reason. Anyway, now I see that this information needs to be included in the processor description. I will make this improvement.
   
   
   
   
   
   
   
   


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



[GitHub] [nifi] davyam edited a comment on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam edited a comment on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020720083


   > I have tested this code.
   > 
   > ✅ Build without exceptions ✅ The mock tests seem enough ✅ The CDC start without problems including PostgreSQL slots ✅ The Syncing Turn On process runs fine ❌ The recovery process, dont conect and recovery from abrutal shutdown of PostgreSQL of NiFi. I beleive the processor stat came be definied before PostgreSQL lsn Commit, so the PostgreSQL cannot resync with the last past LSN. ✅If the stop process, the processor restart from last step without problems. ❌ The connection with a slot previously created don't run.
   > 
   > Congrats for the evolution of this processor code.
   
   Hi man! Thanks for the feedback!
   
   About the recovery test, please give me more information about your simulation. The PostgreSQL Replication Slot doesn't keep changes after the feedback (setAppliedLSN and setFlushedLSN functions), which is performed as soon as the change is received by the processor. Then, if you try to set the Start LSN propriety with an LSN already received, that will not work and the processor will start from the pending changes available in the slot. But you can test the opposite, I mean, you can request an LSN after many unreceived transactions, so the processor will skip them and start from the informed LSN. Take a look at the integration test testCDCStartLSN.
   
   Talking about the test with an existing slot, this is working. I tested it many times, including integration tests. I believe that, previously, you don't create the slot with **pgoutput** output plugin. For example:
   
   `SELECT pg_create_logical_replication_slot('slot_city', 'pgoutput');`
   
   Probably this is the reason. Anyway, now I see that this information needs to be included in the processor description. I will make this improvement.
   
   
   
   
   
   
   
   


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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1024754843


   Hi @joewitt! 
   
   The last build failed because we weren't expecting different time zones in the unit tests (being more specific, xCommitTime in the flowfile content). 
   
   Now, this problem was solved. When possible, please, start the build again. Thanks!


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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795089530



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);
+        }
+    }
+
+    @OnStopped
+    public void onStopped(ProcessContext context) {
+        try {
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    @OnShutdown
+    public void onShutdown(ProcessContext context) {
+        try {
+            // In case we get shutdown while still running, save off the current state,
+            // disconnect, and shut down gracefully.
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    protected void createReplicationReader(String slot, boolean dropSlotIfExists, String publication, Long lsn,
+            boolean includeBeginCommit, boolean includeAllMetadata, Connection replicationConn,
+            Connection queryConn) throws SQLException {
+        this.replicationReader = new Reader(slot, dropSlotIfExists, publication, lsn, includeBeginCommit,
+                includeAllMetadata, replicationConn, queryConn);
+    }
+
+    protected Long getMaxFlowFileListSize() {
+        return this.maxFlowFileListSize;
+    }
+
+    protected void connect(String host, String port, String database, String username, String password,
+            String driverLocation, String driverName, long connectionTimeout)
+            throws IOException, TimeoutException {
+        try {
+            // Ensure driverLocation and driverName are correct
+            // before establishing connection.
+            registerDriver(driverLocation, driverName);
+        } catch (InitializationException e) {
+            throw new RuntimeException(
+                    "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                            + "are configured correctly. " + e,

Review comment:
       I'll include the suggestion in the next commit. Thanks!




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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795073584



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();

Review comment:
       This commented line should be 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.

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

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



[GitHub] [nifi] davyam edited a comment on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam edited a comment on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020539193






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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795087646



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/event/MockReader.java
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.event;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.sql.SQLException;
+import java.text.ParseException;
+import java.util.HashMap;
+
+import org.postgresql.replication.LogSequenceNumber;
+
+/**
+ * Mock Reader class.
+ */
+public class MockReader extends Reader {
+  private boolean includeBeginCommit;
+  private boolean includeAllMetadata;
+  private MockPGReplicationStream replicationStream;
+
+  public MockReader(Long lsn, boolean includeBeginCommit, boolean includeAllMetadata) throws SQLException {
+    super(lsn, includeBeginCommit, includeAllMetadata);
+
+    try {
+      // Creating a mock replication stream
+      LogSequenceNumber startLSN = lsn == null ? LogSequenceNumber.valueOf(100L) : LogSequenceNumber.valueOf(lsn);
+      this.replicationStream = new MockPGReplicationStream(startLSN);
+
+    } catch (SQLException e) {
+      throw new SQLException("Failed to create replication stream. " + e, e);
+    }

Review comment:
       Yeah, I agree, this must be included in the NiFi Checkstyle! Recently I changed this configuration (tab instead four-spaces) for another project. I'll undo and update the code. No problem!




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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795910722



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/event/MockReader.java
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.event;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.sql.SQLException;
+import java.text.ParseException;
+import java.util.HashMap;
+
+import org.postgresql.replication.LogSequenceNumber;
+
+/**
+ * Mock Reader class.
+ */
+public class MockReader extends Reader {
+  private boolean includeBeginCommit;
+  private boolean includeAllMetadata;
+  private MockPGReplicationStream replicationStream;
+
+  public MockReader(Long lsn, boolean includeBeginCommit, boolean includeAllMetadata) throws SQLException {
+    super(lsn, includeBeginCommit, includeAllMetadata);
+
+    try {
+      // Creating a mock replication stream
+      LogSequenceNumber startLSN = lsn == null ? LogSequenceNumber.valueOf(100L) : LogSequenceNumber.valueOf(lsn);
+      this.replicationStream = new MockPGReplicationStream(startLSN);
+
+    } catch (SQLException e) {
+      throw new SQLException("Failed to create replication stream. " + e, e);
+    }

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.

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

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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795912617



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();

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.

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

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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795089219



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();

Review comment:
       All right!




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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795089171



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";
+        private static final String DRIVER_NAME = "org.postgresql.Driver";
+        private static final String HOST = "192.168.56.101";
+        private static final String PORT = "5432";
+        private static final String DATABASE = "db_test";
+        private static final String USERNAME = "nifi";
+        private static final String PASSWORD = "Change1t!";
+        private static final String CONNECTION_TIMEOUT = "30 seconds";
+        private static final String PUBLICATION = "pub_city";
+        private static final String REPLICATION_SLOT = "slot_city";
+
+        ITPostgreSQLClient client;
+        TestRunner runner;
+
+        @Before
+        public void setUp() throws Exception {
+                runner = TestRunners.newTestRunner(new CaptureChangePostgreSQL());
+                client = new ITPostgreSQLClient(HOST, PORT, DATABASE, USERNAME, PASSWORD);
+        }
+
+        @After
+        public void tearDown() throws Exception {
+                runner = null;
+
+                if (client != null) {
+                        client.closeConnection();
+                        client = null;
+                }
+        }
+
+        @Test
+        public void test01RequiredProperties() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+                runner.assertValid();
+        }
+
+        @Test
+        public void test02RegisterDriverFailure() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+                runner.assertValid();
+
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, "/tmp/postgrez.jar"); // Incorrect driver
+                                                                                                  // location
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.assertValid();
+
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, "org.postgrez.Driver"); // Incorrect driver name
+                try {
+                        runner.run();
+                } catch (AssertionError ae) {
+                        Throwable pe = ae.getCause();
+                        assertTrue(pe instanceof ProcessException);
+                        Throwable rue = pe.getCause();
+                        assertTrue(rue instanceof RuntimeException);
+                        assertEquals(
+                                        "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                                                        + "are configured correctly. org.apache.nifi.reporting.InitializationException: Can't load Database Driver",
+                                        rue.getMessage());
+                        Throwable ine = rue.getCause();
+                        assertTrue(ine instanceof InitializationException);
+                        assertEquals("Can't load Database Driver", ine.getMessage());
+                }
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+        }
+
+        @Test
+        public void test03ConnectionFailures() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+
+                runner.setProperty(CaptureChangePostgreSQL.HOST, "192.168.56.199"); // Incorrect IP
+                try {
+                        runner.run();
+                } catch (AssertionError ae) {
+                        Throwable pe = ae.getCause();
+                        assertTrue(pe instanceof ProcessException);
+                        Throwable ioe = pe.getCause();
+                        assertTrue(ioe instanceof IOException);
+                        assertEquals("Error creating SQL connection to specified host and port", ioe.getMessage());
+                        Throwable psqle = ioe.getCause();
+                        assertTrue(psqle instanceof PSQLException);
+                }

Review comment:
       Other processors did the same. I used them as an example. But, no problem, I'll make this improvement in the 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.

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

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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r794971376



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>

Review comment:
       @joewitt No problem man! We'll take a look and do whatever is necessary to comply with this license issue.
   Thank you so much for the feedback and tips!




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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795073834



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);
+        }
+    }
+
+    @OnStopped
+    public void onStopped(ProcessContext context) {
+        try {
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    @OnShutdown
+    public void onShutdown(ProcessContext context) {
+        try {
+            // In case we get shutdown while still running, save off the current state,
+            // disconnect, and shut down gracefully.
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    protected void createReplicationReader(String slot, boolean dropSlotIfExists, String publication, Long lsn,
+            boolean includeBeginCommit, boolean includeAllMetadata, Connection replicationConn,
+            Connection queryConn) throws SQLException {
+        this.replicationReader = new Reader(slot, dropSlotIfExists, publication, lsn, includeBeginCommit,
+                includeAllMetadata, replicationConn, queryConn);
+    }
+
+    protected Long getMaxFlowFileListSize() {
+        return this.maxFlowFileListSize;
+    }
+
+    protected void connect(String host, String port, String database, String username, String password,
+            String driverLocation, String driverName, long connectionTimeout)
+            throws IOException, TimeoutException {
+        try {
+            // Ensure driverLocation and driverName are correct
+            // before establishing connection.
+            registerDriver(driverLocation, driverName);
+        } catch (InitializationException e) {
+            throw new RuntimeException(
+                    "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                            + "are configured correctly. " + e,
+                    e);
+        }
+
+        // Connection expects a non-null password.
+        if (password == null) {
+            password = "";
+        }
+
+        // Connection expects a timeout.
+        if (connectionTimeout == 0) {
+            connectionTimeout = Long.MAX_VALUE;
+        }
+
+        InetSocketAddress address = getAddress(host, port);
+
+        queryConnHolder = new JDBCConnectionHolder(address, database, username, password, false, connectionTimeout);
+        try {
+            // Ensure Query connection can be created.
+            this.getQueryConnection();
+        } catch (SQLException e) {
+            throw new IOException("Error creating SQL connection to specified host and port", e);
+        }
+
+        replicationConnHolder = new JDBCConnectionHolder(address, database, username, password, true,
+                connectionTimeout);
+        try {
+            // Ensure Replication connection can be created.
+            this.getReplicationConnection();
+        } catch (SQLException e) {
+            throw new IOException("Error creating Replication connection to specified host and port", e);
+        }
+    }
+
+    protected Connection getQueryConnection() throws SQLException {
+        return queryConnHolder.getConnection();
+    }
+
+    protected Connection getReplicationConnection() throws SQLException {
+        return replicationConnHolder.getConnection();
+    }
+
+    protected void closeQueryConnection() {
+        if (queryConnHolder != null)
+            queryConnHolder.close();
+    }
+
+    protected void closeReplicationConnection() {
+        if (replicationConnHolder != null)
+            replicationConnHolder.close();

Review comment:
       All blocks should include standard brackets:
   ```suggestion
           if (replicationConnHolder != null) {
               replicationConnHolder.close();
           }
   ```




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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795073763



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);
+        }
+    }
+
+    @OnStopped
+    public void onStopped(ProcessContext context) {
+        try {
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    @OnShutdown
+    public void onShutdown(ProcessContext context) {
+        try {
+            // In case we get shutdown while still running, save off the current state,
+            // disconnect, and shut down gracefully.
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    protected void createReplicationReader(String slot, boolean dropSlotIfExists, String publication, Long lsn,
+            boolean includeBeginCommit, boolean includeAllMetadata, Connection replicationConn,
+            Connection queryConn) throws SQLException {
+        this.replicationReader = new Reader(slot, dropSlotIfExists, publication, lsn, includeBeginCommit,
+                includeAllMetadata, replicationConn, queryConn);
+    }
+
+    protected Long getMaxFlowFileListSize() {
+        return this.maxFlowFileListSize;
+    }
+
+    protected void connect(String host, String port, String database, String username, String password,
+            String driverLocation, String driverName, long connectionTimeout)
+            throws IOException, TimeoutException {
+        try {
+            // Ensure driverLocation and driverName are correct
+            // before establishing connection.
+            registerDriver(driverLocation, driverName);
+        } catch (InitializationException e) {
+            throw new RuntimeException(
+                    "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                            + "are configured correctly. " + e,

Review comment:
       Appending the Exception.toString() seems unnecessary since the exception itself is already included.
   ```suggestion
                               + "are configured correctly",
   ```




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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020553625


   Hey @rrjanbiah @driesva @mathiasbosman, 
   
   We believe that the problems and improvements asked in the last PR #4065 were made. Feel free to check and test. We are counting on your feedback too.


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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795088507



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";

Review comment:
       OK, I'll change the Integration Test to read the value of properties from environment variables.




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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r811532609



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>

Review comment:
       Hey @joewitt, the waffle-jna is not present in mvn dependency tree. Maybe because it is an optional dependency for postgresql lib. Take a look:
   
   $ mvn dependency:tree
   ...
   --- maven-dependency-plugin:3.2.0:tree (default-cli) @ nifi-cdc-postgresql-nar ---
   org.apache.nifi:nifi-cdc-postgresql-nar:nar:1.16.0-SNAPSHOT
   +- org.apache.nifi:nifi-cdc-postgresql-processors:jar:1.16.0-SNAPSHOT:compile
   |  +- org.apache.nifi:nifi-cdc-api:jar:1.16.0-SNAPSHOT:compile
   |  |  \- org.apache.commons:commons-lang3:jar:3.9:compile
   |  +- org.apache.nifi:nifi-utils:jar:1.16.0-SNAPSHOT:compile
   |  |  +- org.apache.nifi:nifi-api:jar:1.16.0-SNAPSHOT:provided
   |  |  +- jakarta.xml.bind:jakarta.xml.bind-api:jar:2.3.3:compile
   |  |  |  \- jakarta.activation:jakarta.activation-api:jar:1.2.2:compile
   |  |  \- org.glassfish.jaxb:jaxb-runtime:jar:2.3.5:compile
   |  |     +- org.glassfish.jaxb:txw2:jar:2.3.5:compile
   |  |     +- com.sun.istack:istack-commons-runtime:jar:3.0.12:compile
   |  |     \- com.sun.activation:jakarta.activation:jar:1.2.2:runtime
   **|  +- org.postgresql:postgresql:jar:42.3.3:compile
   |  |  \- org.checkerframework:checker-qual:jar:3.5.0:runtime**
   |  \- com.fasterxml.jackson.core:jackson-databind:jar:2.12.5:compile
   |     +- com.fasterxml.jackson.core:jackson-annotations:jar:2.12.5:compile
   |     \- com.fasterxml.jackson.core:jackson-core:jar:2.12.5:compile  
   ...
   
   In this case, I added only the postgresql and checker-qual licenses. Is that okay?




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



[GitHub] [nifi] davyam edited a comment on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam edited a comment on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020720083






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



[GitHub] [nifi] joewitt commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
joewitt commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r794961720



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>
+        </dependency>
+        <!-- https://mvnrepository.com/artifact/com.fasterxml.jackson.core/jackson-databind -->
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.13.1</version>

Review comment:
       lets make sure this is consistent with the rest of hte codebase use of jackson-databind as best we can. I've not looked but just flagging as a common challenge.




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



[GitHub] [nifi] joewitt commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
joewitt commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r794964019



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>

Review comment:
       @gerdansantos Yep I know.  I am referring to what it takes to actually use these things in Apache Software Foundation releases.  We need proper license/notice reflections for this.  We have it documented here https://nifi.apache.org/licensing-guide.html.  Take a look at my previous reply.  And find a NOTICE/LICENSE pair in another nar as an example of how we take care of the nar.  The nar LICENSE/NOTICE should reflect precisely what libs are bundled within it.  The LICENSE of the nifi-assembly needs anything not already mentioned added to it for anything that is category A.
   
   This part is admittedly the worst part and the biggest shocker for folks who want to contribute.  Happy to help here but there is legwork.




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



[GitHub] [nifi] gerdansantos commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
gerdansantos commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r794962247



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>

Review comment:
       The PostgreSQL JDBC Driver is distributed under the BSD-2-Clause License. The simplest explanation of the licensing terms is that you can do whatever you want with the product and source code as long as you don't claim you wrote it or sue us. You should give it a read though, it's only half a page.
   
   Copyright (c) 1997, PostgreSQL Global Development Group
   All rights reserved.
   
   Redistribution and use in source and binary forms, with or without
   modification, are permitted provided that the following conditions are met:
   
   1. Redistributions of source code must retain the above copyright notice,
      this list of conditions and the following disclaimer.
   2. Redistributions in binary form must reproduce the above copyright notice,
      this list of conditions and the following disclaimer in the documentation
      and/or other materials provided with the distribution.
   
   THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
   AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
   IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
   ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
   LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
   CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
   SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
   INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
   CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
   ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
   POSSIBILITY OF SUCH DAMAGE.




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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1026044085


   > Thanks for picking up and working on this feature @davyam! I noted several areas for improvement in terms of code formatting.
   
   Hi @exceptionfactory! The improvements were made. Thanks for the feedback and suggestions!


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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795089730



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);
+        }
+    }
+
+    @OnStopped
+    public void onStopped(ProcessContext context) {
+        try {
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    @OnShutdown
+    public void onShutdown(ProcessContext context) {
+        try {
+            // In case we get shutdown while still running, save off the current state,
+            // disconnect, and shut down gracefully.
+            stop(context.getStateManager());
+        } catch (CDCException ioe) {
+            throw new ProcessException(ioe);
+        }
+    }
+
+    protected void createReplicationReader(String slot, boolean dropSlotIfExists, String publication, Long lsn,
+            boolean includeBeginCommit, boolean includeAllMetadata, Connection replicationConn,
+            Connection queryConn) throws SQLException {
+        this.replicationReader = new Reader(slot, dropSlotIfExists, publication, lsn, includeBeginCommit,
+                includeAllMetadata, replicationConn, queryConn);
+    }
+
+    protected Long getMaxFlowFileListSize() {
+        return this.maxFlowFileListSize;
+    }
+
+    protected void connect(String host, String port, String database, String username, String password,
+            String driverLocation, String driverName, long connectionTimeout)
+            throws IOException, TimeoutException {
+        try {
+            // Ensure driverLocation and driverName are correct
+            // before establishing connection.
+            registerDriver(driverLocation, driverName);
+        } catch (InitializationException e) {
+            throw new RuntimeException(
+                    "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                            + "are configured correctly. " + e,
+                    e);
+        }
+
+        // Connection expects a non-null password.
+        if (password == null) {
+            password = "";
+        }
+
+        // Connection expects a timeout.
+        if (connectionTimeout == 0) {
+            connectionTimeout = Long.MAX_VALUE;
+        }
+
+        InetSocketAddress address = getAddress(host, port);
+
+        queryConnHolder = new JDBCConnectionHolder(address, database, username, password, false, connectionTimeout);
+        try {
+            // Ensure Query connection can be created.
+            this.getQueryConnection();
+        } catch (SQLException e) {
+            throw new IOException("Error creating SQL connection to specified host and port", e);
+        }
+
+        replicationConnHolder = new JDBCConnectionHolder(address, database, username, password, true,
+                connectionTimeout);
+        try {
+            // Ensure Replication connection can be created.
+            this.getReplicationConnection();
+        } catch (SQLException e) {
+            throw new IOException("Error creating Replication connection to specified host and port", e);
+        }
+    }
+
+    protected Connection getQueryConnection() throws SQLException {
+        return queryConnHolder.getConnection();
+    }
+
+    protected Connection getReplicationConnection() throws SQLException {
+        return replicationConnHolder.getConnection();
+    }
+
+    protected void closeQueryConnection() {
+        if (queryConnHolder != null)
+            queryConnHolder.close();
+    }
+
+    protected void closeReplicationConnection() {
+        if (replicationConnHolder != null)
+            replicationConnHolder.close();

Review comment:
       OK! No problem! I'll include the suggestion in the next commit. Maybe, if possible, this check must be in Checkstyle too.




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



[GitHub] [nifi] gerdansantos commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
gerdansantos commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020644890


   I have tested this code.
   
   ✅ Build without exceptions
   ✅ The mock tests seem enough
   ✅ The CDC start without problems including PostgreSQL slots
   ✅ The Syncing Turn On process runs fine
   ❌ The recovery process, dont conect and recovery from abrutal shutdown of PostgreSQL of NiFi. I beleive the processor stat came be definied before PostgreSQL lsn Commit, so the PostgreSQL cannot resync with the last past LSN.
   ✅If the stop process, the processor restart from last step without problems.
   ❌ The connection with a slot previously created don't run.
   
   
   Congrats for the evolution of this processor code.
   
   


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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r811532609



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>

Review comment:
       Hey @joewitt, the waffle-jna is not present in mvn dependency tree. Maybe because it is an optional dependency for postgresql lib. Take a look:
   
   $ mvn dependency:tree
   ...
   --- maven-dependency-plugin:3.2.0:tree (default-cli) @ nifi-cdc-postgresql-nar ---
   org.apache.nifi:nifi-cdc-postgresql-nar:nar:1.16.0-SNAPSHOT
   +- org.apache.nifi:nifi-cdc-postgresql-processors:jar:1.16.0-SNAPSHOT:compile
   |  +- org.apache.nifi:nifi-cdc-api:jar:1.16.0-SNAPSHOT:compile
   |  |  \- org.apache.commons:commons-lang3:jar:3.9:compile
   |  +- org.apache.nifi:nifi-utils:jar:1.16.0-SNAPSHOT:compile
   |  |  +- org.apache.nifi:nifi-api:jar:1.16.0-SNAPSHOT:provided
   |  |  +- jakarta.xml.bind:jakarta.xml.bind-api:jar:2.3.3:compile
   |  |  |  \- jakarta.activation:jakarta.activation-api:jar:1.2.2:compile
   |  |  \- org.glassfish.jaxb:jaxb-runtime:jar:2.3.5:compile
   |  |     +- org.glassfish.jaxb:txw2:jar:2.3.5:compile
   |  |     +- com.sun.istack:istack-commons-runtime:jar:3.0.12:compile
   |  |     \- com.sun.activation:jakarta.activation:jar:1.2.2:runtime
   **|  +- org.postgresql:postgresql:jar:42.3.3:compile
   |  |  \- org.checkerframework:checker-qual:jar:3.5.0:runtime**
   |  \- com.fasterxml.jackson.core:jackson-databind:jar:2.12.5:compile
   |     +- com.fasterxml.jackson.core:jackson-annotations:jar:2.12.5:compile
   |     \- com.fasterxml.jackson.core:jackson-core:jar:2.12.5:compile  
   ...
   
   In this case, I added only the postgresql and checker-qual licenses. Is this okay?

##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>
+        </dependency>
+        <!-- https://mvnrepository.com/artifact/com.fasterxml.jackson.core/jackson-databind -->
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.13.1</version>

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.

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

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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795911970



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";

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.

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

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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795072562



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/event/MockReader.java
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.event;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.sql.SQLException;
+import java.text.ParseException;
+import java.util.HashMap;
+
+import org.postgresql.replication.LogSequenceNumber;
+
+/**
+ * Mock Reader class.
+ */
+public class MockReader extends Reader {
+  private boolean includeBeginCommit;
+  private boolean includeAllMetadata;
+  private MockPGReplicationStream replicationStream;
+
+  public MockReader(Long lsn, boolean includeBeginCommit, boolean includeAllMetadata) throws SQLException {
+    super(lsn, includeBeginCommit, includeAllMetadata);
+
+    try {
+      // Creating a mock replication stream
+      LogSequenceNumber startLSN = lsn == null ? LogSequenceNumber.valueOf(100L) : LogSequenceNumber.valueOf(lsn);
+      this.replicationStream = new MockPGReplicationStream(startLSN);
+
+    } catch (SQLException e) {
+      throw new SQLException("Failed to create replication stream. " + e, e);
+    }

Review comment:
       The formatting in this file does not appear to follow the standard conventions of four-spaces for indentation.  It looks like we may need to adjust the Checkstyle configuration to ensure proper formatting, but this file should be reformatted using standard four-spaces for indentation.




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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020539193






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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1024199981


   @joewitt @pvillard31 @mattyb149 
   
   Hey guys, please, fire off again the workflow. Thanks!


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



[GitHub] [nifi] davyam edited a comment on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam edited a comment on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020720083


   > I have tested this code.
   > 
   > ✅ Build without exceptions ✅ The mock tests seem enough ✅ The CDC start without problems including PostgreSQL slots ✅ The Syncing Turn On process runs fine ❌ The recovery process, dont conect and recovery from abrutal shutdown of PostgreSQL of NiFi. I beleive the processor stat came be definied before PostgreSQL lsn Commit, so the PostgreSQL cannot resync with the last past LSN. ✅If the stop process, the processor restart from last step without problems. ❌ The connection with a slot previously created don't run.
   > 
   > Congrats for the evolution of this processor code.
   
   Hi man! Thanks for the feedback!
   
   About the recovery test, please give me more information about your simulation. The PostgreSQL Replication Slot doesn't keep changes after the feedback (setAppliedLSN and setFlushedLSN functions), which is performed as soon as the change is received by the processor. Then, if you try to set the Start LSN propriety with an LSN already received, that will not work and the processor will start from the pending changes available in the slot. But you can test the opposite, I mean, you can request an LSN after many unreceived transactions, so the processor will skip them and start from the informed LSN (since the state is clean). Take a look at the integration test testCDCStartLSN.
   
   Talking about the test with an existing slot, this is working. I tested it many times, including integration tests. I believe that, previously, you don't create the slot with **pgoutput** output plugin. For example:
   
   `SELECT pg_create_logical_replication_slot('slot_city', 'pgoutput');`
   
   Probably this is the reason. Anyway, now I see that this information needs to be included in the processor description. I will make this improvement.
   
   
   
   
   
   
   
   


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



[GitHub] [nifi] joewitt commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
joewitt commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020569739


   Fired off the workflow run for new contributor (thanks!) and closed the old referenced PR as requested. Hopoefully this one makes quick progress into a merge/release.
   
   Thanks


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



[GitHub] [nifi] gerdansantos commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
gerdansantos commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1024617409


   I have tested this code.
   
   ✅ Build without exceptions ✅ The mock tests seem enough ✅ The CDC start without problems including PostgreSQL slots ✅ The Syncing Turn On process runs fine ✅ The recovery process, dont conect and recovery from abrutal shutdown of PostgreSQL of NiFi. I beleive the processor stat came be definied before PostgreSQL lsn Commit, so the PostgreSQL cannot resync with the last past LSN. ✅If the stop process, the processor restart from last step without problems. ✅ The connection with a slot previously created don't run.
   
   @joewitt @mattyb149 please go on with ci/cd flow!!!


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



[GitHub] [nifi] joewitt commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
joewitt commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r794961823



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>

Review comment:
       What is the license for postgres lib and what transitive deps does this pull in?  We need to account for these in the License and Notice as appropriate.  If this part is easy it means it hasn't been done yet.




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



[GitHub] [nifi] davyam edited a comment on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam edited a comment on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1024972496


   Hey @exceptionfactory! Thanks for the feedback! I'll take a look at the suggestions.


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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1024972496


   Hey @exceptionfactory! Thanks for the feedback! I'll take a look.


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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795104905



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";
+        private static final String DRIVER_NAME = "org.postgresql.Driver";
+        private static final String HOST = "192.168.56.101";
+        private static final String PORT = "5432";
+        private static final String DATABASE = "db_test";
+        private static final String USERNAME = "nifi";
+        private static final String PASSWORD = "Change1t!";
+        private static final String CONNECTION_TIMEOUT = "30 seconds";
+        private static final String PUBLICATION = "pub_city";
+        private static final String REPLICATION_SLOT = "slot_city";
+
+        ITPostgreSQLClient client;
+        TestRunner runner;
+
+        @Before
+        public void setUp() throws Exception {
+                runner = TestRunners.newTestRunner(new CaptureChangePostgreSQL());
+                client = new ITPostgreSQLClient(HOST, PORT, DATABASE, USERNAME, PASSWORD);
+        }
+
+        @After
+        public void tearDown() throws Exception {
+                runner = null;
+
+                if (client != null) {
+                        client.closeConnection();
+                        client = null;
+                }
+        }
+
+        @Test
+        public void test01RequiredProperties() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+                runner.assertValid();
+        }
+
+        @Test
+        public void test02RegisterDriverFailure() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+                runner.assertValid();
+
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, "/tmp/postgrez.jar"); // Incorrect driver
+                                                                                                  // location
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.assertValid();
+
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, "org.postgrez.Driver"); // Incorrect driver name
+                try {
+                        runner.run();
+                } catch (AssertionError ae) {
+                        Throwable pe = ae.getCause();
+                        assertTrue(pe instanceof ProcessException);
+                        Throwable rue = pe.getCause();
+                        assertTrue(rue instanceof RuntimeException);
+                        assertEquals(
+                                        "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                                                        + "are configured correctly. org.apache.nifi.reporting.InitializationException: Can't load Database Driver",
+                                        rue.getMessage());
+                        Throwable ine = rue.getCause();
+                        assertTrue(ine instanceof InitializationException);
+                        assertEquals("Can't load Database Driver", ine.getMessage());
+                }
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+        }
+
+        @Test
+        public void test03ConnectionFailures() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+
+                runner.setProperty(CaptureChangePostgreSQL.HOST, "192.168.56.199"); // Incorrect IP
+                try {
+                        runner.run();
+                } catch (AssertionError ae) {
+                        Throwable pe = ae.getCause();
+                        assertTrue(pe instanceof ProcessException);
+                        Throwable ioe = pe.getCause();
+                        assertTrue(ioe instanceof IOException);
+                        assertEquals("Error creating SQL connection to specified host and port", ioe.getMessage());
+                        Throwable psqle = ioe.getCause();
+                        assertTrue(psqle instanceof PSQLException);
+                }

Review comment:
       There is a mix of approaches right now, newer tests leverage the `assertThrows()` method, while older tests still have the try-catch approach.  Thanks for working through the changes!




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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1024517839


   Hi @gerdansantos! As you suggested, the replication feedback now occurs after the flowfiles transfer to success relation completes.


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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795073185



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";
+        private static final String DRIVER_NAME = "org.postgresql.Driver";
+        private static final String HOST = "192.168.56.101";
+        private static final String PORT = "5432";
+        private static final String DATABASE = "db_test";
+        private static final String USERNAME = "nifi";
+        private static final String PASSWORD = "Change1t!";
+        private static final String CONNECTION_TIMEOUT = "30 seconds";
+        private static final String PUBLICATION = "pub_city";
+        private static final String REPLICATION_SLOT = "slot_city";
+
+        ITPostgreSQLClient client;
+        TestRunner runner;
+
+        @Before
+        public void setUp() throws Exception {
+                runner = TestRunners.newTestRunner(new CaptureChangePostgreSQL());
+                client = new ITPostgreSQLClient(HOST, PORT, DATABASE, USERNAME, PASSWORD);
+        }
+
+        @After
+        public void tearDown() throws Exception {
+                runner = null;
+
+                if (client != null) {
+                        client.closeConnection();
+                        client = null;
+                }
+        }

Review comment:
       The indentation formatting for this file also appears incorrect and needs to be changed to four spaces.




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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1021118807


   > > Fired off the workflow run for new contributor (thanks!) and closed the old referenced PR as requested. Hopoefully this one makes quick progress into a merge/release.
   > > Thanks
   > 
   > @joewitt
   > 
   > The unit tests are failing because they are trying to validate the driver location. I will fix this, leaving this validation only in the integration tests.
   
   Hi @joewitt! The driver location validation was removed from Unit Tests. Please, fire off the workflow again. Thanks!


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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020724700


   > Fired off the workflow run for new contributor (thanks!) and closed the old referenced PR as requested. Hopoefully this one makes quick progress into a merge/release.
   > 
   > Thanks
   
   @joewitt 
   
   The unit tests are failing because they are trying to validate the driver location. I will fix this, leaving this validation only in the integration 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.

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

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



[GitHub] [nifi] gerdansantos commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
gerdansantos commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020644890


   I have tested this code.
   
   ✅ Build without exceptions
   ✅ The mock tests seem enough
   ✅ The CDC start without problems including PostgreSQL slots
   ✅ The Syncing Turn On process runs fine
   ❌ The recovery process, dont conect and recovery from abrutal shutdown of PostgreSQL of NiFi. I beleive the processor stat came be definied before PostgreSQL lsn Commit, so the PostgreSQL cannot resync with the last past LSN.
   ✅If the stop process, the processor restart from last step without problems.
   ❌ The connection with a slot previously created don't run.
   
   
   Congrats for the evolution of this processor code.
   
   


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



[GitHub] [nifi] davyam edited a comment on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam edited a comment on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020539193


   Hi @pvillard31 and @mattyb149! This is a new version of CaptureChangePostgreSQL processor.
   
   The code was refactored and many improvements were made. I really appreciate your feedback!


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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020539193


   @pvillard31 and @mattyb149 this is a new version of CaptureChangePostgreSQL processor.
   
   The code was refactored and many improvements were made. I really appreciate your feedback!


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



[GitHub] [nifi] joewitt commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
joewitt commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020569739


   Fired off the workflow run for new contributor (thanks!) and closed the old referenced PR as requested. Hopoefully this one makes quick progress into a merge/release.
   
   Thanks


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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795913390



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);

Review comment:
       Done! Suggestion included in last commit. Thanks!




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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795073247



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";
+        private static final String DRIVER_NAME = "org.postgresql.Driver";
+        private static final String HOST = "192.168.56.101";
+        private static final String PORT = "5432";
+        private static final String DATABASE = "db_test";
+        private static final String USERNAME = "nifi";
+        private static final String PASSWORD = "Change1t!";
+        private static final String CONNECTION_TIMEOUT = "30 seconds";
+        private static final String PUBLICATION = "pub_city";
+        private static final String REPLICATION_SLOT = "slot_city";
+
+        ITPostgreSQLClient client;
+        TestRunner runner;
+
+        @Before
+        public void setUp() throws Exception {
+                runner = TestRunners.newTestRunner(new CaptureChangePostgreSQL());
+                client = new ITPostgreSQLClient(HOST, PORT, DATABASE, USERNAME, PASSWORD);
+        }
+
+        @After
+        public void tearDown() throws Exception {
+                runner = null;
+
+                if (client != null) {
+                        client.closeConnection();
+                        client = null;
+                }
+        }
+
+        @Test
+        public void test01RequiredProperties() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+                runner.assertValid();
+        }
+
+        @Test
+        public void test02RegisterDriverFailure() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+                runner.assertValid();
+
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, "/tmp/postgrez.jar"); // Incorrect driver
+                                                                                                  // location
+                runner.assertNotValid();
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.assertValid();
+
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, "org.postgrez.Driver"); // Incorrect driver name
+                try {
+                        runner.run();
+                } catch (AssertionError ae) {
+                        Throwable pe = ae.getCause();
+                        assertTrue(pe instanceof ProcessException);
+                        Throwable rue = pe.getCause();
+                        assertTrue(rue instanceof RuntimeException);
+                        assertEquals(
+                                        "Failed to register JDBC driver. Ensure PostgreSQL Driver Location(s) and PostgreSQL Driver Class Name "
+                                                        + "are configured correctly. org.apache.nifi.reporting.InitializationException: Can't load Database Driver",
+                                        rue.getMessage());
+                        Throwable ine = rue.getCause();
+                        assertTrue(ine instanceof InitializationException);
+                        assertEquals("Can't load Database Driver", ine.getMessage());
+                }
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+        }
+
+        @Test
+        public void test03ConnectionFailures() {
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_LOCATION, DRIVER_LOCATION);
+                runner.setProperty(CaptureChangePostgreSQL.DRIVER_NAME, DRIVER_NAME);
+                runner.setProperty(CaptureChangePostgreSQL.HOST, HOST);
+                runner.setProperty(CaptureChangePostgreSQL.PORT, PORT);
+                runner.setProperty(CaptureChangePostgreSQL.DATABASE, DATABASE);
+                runner.setProperty(CaptureChangePostgreSQL.USERNAME, USERNAME);
+                runner.setProperty(CaptureChangePostgreSQL.PASSWORD, PASSWORD);
+                runner.setProperty(CaptureChangePostgreSQL.CONNECTION_TIMEOUT, CONNECTION_TIMEOUT);
+                runner.setProperty(CaptureChangePostgreSQL.PUBLICATION, PUBLICATION);
+                runner.setProperty(CaptureChangePostgreSQL.REPLICATION_SLOT, REPLICATION_SLOT);
+
+                runner.setProperty(CaptureChangePostgreSQL.HOST, "192.168.56.199"); // Incorrect IP
+                try {
+                        runner.run();
+                } catch (AssertionError ae) {
+                        Throwable pe = ae.getCause();
+                        assertTrue(pe instanceof ProcessException);
+                        Throwable ioe = pe.getCause();
+                        assertTrue(ioe instanceof IOException);
+                        assertEquals("Error creating SQL connection to specified host and port", ioe.getMessage());
+                        Throwable psqle = ioe.getCause();
+                        assertTrue(psqle instanceof PSQLException);
+                }

Review comment:
       This type of evaluation should be changed to use JUnit `assertThrows()` with the expected exception as the first parameter.




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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795073146



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";

Review comment:
       This should be changed to read from a System property for environment variable as opposed to hard-coding a particular local path.  The same applies to the other properties.




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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795088978



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";
+        private static final String DRIVER_NAME = "org.postgresql.Driver";
+        private static final String HOST = "192.168.56.101";
+        private static final String PORT = "5432";
+        private static final String DATABASE = "db_test";
+        private static final String USERNAME = "nifi";
+        private static final String PASSWORD = "Change1t!";
+        private static final String CONNECTION_TIMEOUT = "30 seconds";
+        private static final String PUBLICATION = "pub_city";
+        private static final String REPLICATION_SLOT = "slot_city";
+
+        ITPostgreSQLClient client;
+        TestRunner runner;
+
+        @Before
+        public void setUp() throws Exception {
+                runner = TestRunners.newTestRunner(new CaptureChangePostgreSQL());
+                client = new ITPostgreSQLClient(HOST, PORT, DATABASE, USERNAME, PASSWORD);
+        }
+
+        @After
+        public void tearDown() throws Exception {
+                runner = null;
+
+                if (client != null) {
+                        client.closeConnection();
+                        client = null;
+                }
+        }

Review comment:
       The same before. I'll update the indentation to four spaces.




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



[GitHub] [nifi] joewitt commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
joewitt commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1024618768


   The build is in progress but all the comments and testing and general contribution here is really awesome to see. Thanks to you all


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



[GitHub] [nifi] joewitt commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
joewitt commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r794963393



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>

Review comment:
       Looks like it is https://github.com/pgjdbc/pgjdbc/blob/master/LICENSE. We'll need that in the LICENSE for this nar file (see other nars for examples) and we'll need this in the LICENSE of the nifi-assembly.   We don't need it in the top level LICENSE of the entire source tree since this is a binary only dependency.  There are these two immediate dependencies
   
   <groupId>org.checkerframework</groupId>
   <artifactId>checker-qual</artifactId>
   <version>3.5.0</version>
   <scope>runtime</scope>
   </dependency>
   <dependency>
   <groupId>com.github.waffle</groupId>
   <artifactId>waffle-jna</artifactId>
   <version>1.9.1</version>
   <optional>true</optional>
   
   Both appear to be MIT so they too would go in the mentioned LICENSES.  But jna appears to have several deps that also need to be followed up on.




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



[GitHub] [nifi] davyam commented on pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication.

Posted by GitBox <gi...@apache.org>.
davyam commented on pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#issuecomment-1020720083


   > I have tested this code.
   > 
   > ✅ Build without exceptions ✅ The mock tests seem enough ✅ The CDC start without problems including PostgreSQL slots ✅ The Syncing Turn On process runs fine ❌ The recovery process, dont conect and recovery from abrutal shutdown of PostgreSQL of NiFi. I beleive the processor stat came be definied before PostgreSQL lsn Commit, so the PostgreSQL cannot resync with the last past LSN. ✅If the stop process, the processor restart from last step without problems. ❌ The connection with a slot previously created don't run.
   > 
   > Congrats for the evolution of this processor code.
   
   Hi man! Thanks for the feedback!
   
   About the recovery test, please give me more information about your simulation. The PostgreSQL Replication Slot doesn't keep changes after the feedback (setAppliedLSN and setFlushedLSN functions), which is performed as soon as the change is received by the processor. Then, if you try to set the Start LSN propriety with an LSN already received, that will not work and the processor will start from the pending changes available in the slot. But you can test the opposite, I mean, you can request an LSN after many unreceived transactions, so the processor will skip them. Take a look at the integration test testCDCStartLSN.
   
   Talking about the test with an existing slot, this is working. I tested it many times, including integration tests. I believe that, previously, you don't create the slot with **pgoutput** output plugin. For example:
   
   `SELECT pg_create_logical_replication_slot('slot_city', 'pgoutput');`
   
   Probably this is the reason. Anyway, now I see that this information needs to be included in the processor description. I will make this improvement.
   
   
   
   
   
   
   
   


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



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795072212



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>
+        </dependency>
+        <!-- https://mvnrepository.com/artifact/com.fasterxml.jackson.core/jackson-databind -->
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <version>2.13.1</version>

Review comment:
       This should use the `${jackson.version}` property to inherit the version from the root Maven configuration.




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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795912138



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";
+        private static final String DRIVER_NAME = "org.postgresql.Driver";
+        private static final String HOST = "192.168.56.101";
+        private static final String PORT = "5432";
+        private static final String DATABASE = "db_test";
+        private static final String USERNAME = "nifi";
+        private static final String PASSWORD = "Change1t!";
+        private static final String CONNECTION_TIMEOUT = "30 seconds";
+        private static final String PUBLICATION = "pub_city";
+        private static final String REPLICATION_SLOT = "slot_city";
+
+        ITPostgreSQLClient client;
+        TestRunner runner;
+
+        @Before
+        public void setUp() throws Exception {
+                runner = TestRunners.newTestRunner(new CaptureChangePostgreSQL());
+                client = new ITPostgreSQLClient(HOST, PORT, DATABASE, USERNAME, PASSWORD);
+        }
+
+        @After
+        public void tearDown() throws Exception {
+                runner = null;
+
+                if (client != null) {
+                        client.closeConnection();
+                        client = null;
+                }
+        }

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.

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

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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r794971376



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/pom.xml
##########
@@ -0,0 +1,74 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-cdc-postgresql-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-cdc-postgresql-processors</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-cdc-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-simple</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <!-- https://mvnrepository.com/artifact/org.postgresql/postgresql -->
+        <dependency>
+            <groupId>org.postgresql</groupId>
+            <artifactId>postgresql</artifactId>
+            <version>42.3.1</version>

Review comment:
       @joewitt No problem man! We'll take a look and and do whatever is necessary to comply with this license issue.
   Thank you so much for the feedback and tips!




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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795089266



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/main/java/org/apache/nifi/cdc/postgresql/processors/CaptureChangePostgreSQL.java
##########
@@ -0,0 +1,697 @@
+/*
+ * 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.
+ */
+
+package org.apache.nifi.cdc.postgresql.processors;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.nio.charset.StandardCharsets;
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.DriverPropertyInfo;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.logging.Logger;
+
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.cdc.CDCException;
+import org.apache.nifi.cdc.postgresql.event.Reader;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.resource.ResourceCardinality;
+import org.apache.nifi.components.resource.ResourceType;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.components.state.StateManager;
+import org.apache.nifi.components.state.StateMap;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
+
+/**
+ * A processor to Capture Data Change (CDC) events from a PostgreSQL database
+ * and return them as flow files.
+ */
+@TriggerSerially
+@PrimaryNodeOnly
+@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
+
+@Tags({ "sql", "jdbc", "cdc", "postgresql" })
+
+@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a PostgreSQL database. Works for PostgreSQL version 10+. "
+        + "Events include INSERT, UPDATE, and DELETE operations and are output as individual flow files ordered by the time at which the operation occurred. "
+        + "This processor uses Replication Connection to stream data. By default, an existing Logical Replication Slot with the specified name will be used or, "
+        + "if none exists, a new one will be created. In the case of an existing slot, make sure that pgoutput is the output plugin. "
+        + "This processor also uses SQL Connection to query system views. "
+        + "Furthermore, a Publication in PostgreSQL database should already exist.")
+
+@Stateful(scopes = Scope.CLUSTER, description = "The last received Log Sequence Number (LSN) from Replication Slot is stored by this processor, "
+        + "such that it can continue from the same location if restarted.")
+
+@WritesAttributes({
+        @WritesAttribute(attribute = "cdc.type", description = "The CDC event type, as begin, commit, insert, update, delete, etc."),
+        @WritesAttribute(attribute = "cdc.lsn", description = "The Log Sequence Number (i.e. strictly increasing integer value) specifying the order "
+                + "of the CDC event flow file relative to the other event flow files."),
+        @WritesAttribute(attribute = "mime.type", description = "The processor outputs flow file content in JSON format, and sets the mime.type attribute to application/json.") })
+
+public class CaptureChangePostgreSQL extends AbstractProcessor {
+    // Constants
+    private static final String MIME_TYPE_DEFAULT = "application/json";
+    private static final String LAST_LSN_STATE_MAP_KEY = "last.received.lsn";
+    private static final String POSTGRESQL_MIN_VERSION = "10";
+    private static final String POSTGRESQL_PORT_DEFAULT = "5432";
+
+    // Relationships
+    public static final Relationship REL_SUCCESS = new Relationship.Builder().name("success")
+            .description("Successfully created FlowFile from CDC event.").build();
+
+    private Set<Relationship> relationships;
+
+    // Properties
+    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-class")
+            .displayName("PostgreSQL Driver Class Name")
+            .description("The class name of the PostgreSQL database driver class")
+            .defaultValue("org.postgresql.Driver")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-driver-locations")
+            .displayName("PostgreSQL Driver Location(s)")
+            .description(
+                    "Comma-separated list of files/folders containing the PostgreSQL driver JAR and its dependencies (if any). For example '/var/tmp/postgresql-42.3.1.jar'")
+            .defaultValue(null)
+            .required(false)
+            .identifiesExternalResource(ResourceCardinality.MULTIPLE, ResourceType.FILE)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HOST = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-host")
+            .displayName("PostgreSQL Hostname")
+            .description("The hostname for PostgreSQL connections.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-port")
+            .displayName("PostgreSQL Port")
+            .description("The default port for PostgreSQL connections.")
+            .required(true)
+            .defaultValue(POSTGRESQL_PORT_DEFAULT)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DATABASE = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-database")
+            .displayName("Database")
+            .description("Specifies the name of the database to connect to.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-user")
+            .displayName("Username")
+            .description("Username to access the PostgreSQL database. Must be a superuser.")
+            .required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-password")
+            .displayName("Password")
+            .description("Password to access the PostgreSQL database.")
+            .required(false)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor CONNECTION_TIMEOUT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-max-wait-time").displayName("Max Wait Time")
+            .description(
+                    "The maximum amount of time allowed for a connection to be established, zero means there is effectively no limit.")
+            .defaultValue("30 seconds")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor PUBLICATION = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-publication")
+            .displayName("Publication Name")
+            .description(
+                    "A group of tables whose data changes are intended to be replicated through Logical Replication. "
+                            + "It should be created in the database before the processor starts.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor REPLICATION_SLOT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-slot-name")
+            .displayName("Replication Slot Name")
+            .description(
+                    "A unique, cluster-wide identifier for the PostgreSQL Replication Slot. "
+                            + "If it already exists, make sure that pgoutput is the output plugin.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor DROP_SLOT_IF_EXISTS = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-drop-slot-if-exists")
+            .displayName("Drop if exists replication slot?")
+            .description(
+                    "Drop the Replication Slot in PostgreSQL database if it already exists every processor starts.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_BEGIN_COMMIT = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-begin-commit")
+            .displayName("Include BEGIN and COMMIT statements?")
+            .description(
+                    "Specifies whether to emit events including BEGIN and COMMIT statements. Set to true if the BEGIN and COMMIT statements are necessary in the downstream flow, "
+                            + "otherwise set to false, which suppresses these statements and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor INCLUDE_ALL_METADATA = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-include-all-metadata")
+            .displayName("Include all metadata?")
+            .description(
+                    "Specifies whether to emit events including all message types (BEGIN, COMMIT, RELATION, TYPE, etc) and all metadata (relation id, tuple type, tuple data before update, etc). "
+                            + "Set to true if all metadata are necessary in the downstream flow, otherwise set to false, which suppresses these metadata and can increase flow performance.")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("false")
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor START_LSN = new PropertyDescriptor.Builder()
+            .name("cdc-postgresql-start-lsn")
+            .displayName("Start Log Sequence Number (LSN)")
+            .description(
+                    "Specifies a start Log Sequence Number (LSN) to use if this processor's state does not have a current "
+                            + "sequence identifier. If a LSN is present in the processor's state, this property is ignored.")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR)
+            .build();
+
+    // Key Attributes
+    private List<PropertyDescriptor> descriptors;
+    private JDBCConnectionHolder queryConnHolder = null;
+    private JDBCConnectionHolder replicationConnHolder = null;
+    private volatile Long lastLSN = null;
+    protected Reader replicationReader = null;
+    protected Long maxFlowFileListSize = 100L;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(DRIVER_NAME);
+        descriptors.add(DRIVER_LOCATION);
+        descriptors.add(HOST);
+        descriptors.add(PORT);
+        descriptors.add(DATABASE);
+        descriptors.add(USERNAME);
+        descriptors.add(PASSWORD);
+        descriptors.add(CONNECTION_TIMEOUT);
+        descriptors.add(PUBLICATION);
+        descriptors.add(REPLICATION_SLOT);
+        descriptors.add(DROP_SLOT_IF_EXISTS);
+        descriptors.add(INCLUDE_BEGIN_COMMIT);
+        descriptors.add(INCLUDE_ALL_METADATA);
+        descriptors.add(START_LSN);
+
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_SUCCESS);
+
+        this.relationships = Collections.unmodifiableSet(relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    // Runs the initial configuration of processor.
+    public void setup(ProcessContext context) {
+        final ComponentLog logger = getLogger();
+
+        final StateManager stateManager = context.getStateManager();
+        final StateMap stateMap;
+
+        try {
+            stateMap = stateManager.getState(Scope.CLUSTER);
+        } catch (final IOException ioe) {
+            logger.error(
+                    "Failed to retrieve observed maximum values from the State Manager. Will not attempt connection until this is accomplished.",
+                    ioe);
+            context.yield();
+            return;
+        }
+
+        final String driverName = context.getProperty(DRIVER_NAME).evaluateAttributeExpressions().getValue();
+        final String driverLocation = context.getProperty(DRIVER_LOCATION).evaluateAttributeExpressions().getValue();
+
+        final String host = context.getProperty(HOST).evaluateAttributeExpressions().getValue();
+        final String port = context.getProperty(PORT).evaluateAttributeExpressions().getValue();
+        final String database = context.getProperty(DATABASE).evaluateAttributeExpressions().getValue();
+        final String username = context.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
+        final Long connectionTimeout = context.getProperty(CONNECTION_TIMEOUT).evaluateAttributeExpressions()
+                .asTimePeriod(TimeUnit.MILLISECONDS);
+
+        final String publication = context.getProperty(PUBLICATION).evaluateAttributeExpressions().getValue();
+        final String slot = context.getProperty(REPLICATION_SLOT).evaluateAttributeExpressions().getValue();
+        final Boolean dropSlotIfExists = context.getProperty(DROP_SLOT_IF_EXISTS).asBoolean();
+        final Boolean includeBeginCommit = context.getProperty(INCLUDE_BEGIN_COMMIT).asBoolean();
+        final Boolean includeAllMetadata = context.getProperty(INCLUDE_ALL_METADATA).asBoolean();
+        final Long startLSN = context.getProperty(START_LSN).asLong();
+
+        if (stateMap.get(LAST_LSN_STATE_MAP_KEY) != null && Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY)) > 0L) {
+            this.lastLSN = Long.parseLong(stateMap.get(LAST_LSN_STATE_MAP_KEY));
+        } else {
+            if (startLSN != null)
+                this.lastLSN = startLSN;
+        }
+
+        try {
+            this.connect(host, port, database, username, password, driverLocation, driverName, connectionTimeout);
+            this.createReplicationReader(slot, dropSlotIfExists, publication, lastLSN, includeBeginCommit,
+                    includeAllMetadata, this.getReplicationConnection(), this.getQueryConnection());
+
+        } catch (final IOException | TimeoutException | SQLException | RuntimeException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        // final ComponentLog logger = getLogger();
+
+        if (this.replicationReader == null || this.replicationReader.getReplicationStream() == null) {
+            setup(context);
+        }
+
+        try {
+            List<FlowFile> listFlowFiles = new ArrayList<>();
+
+            while (listFlowFiles.size() < this.getMaxFlowFileListSize()) {
+                HashMap<String, Object> message = this.replicationReader.readMessage();
+
+                if (message == null) // No more messages.
+                    break;
+
+                if (message.isEmpty()) // Skip empty messages.
+                    continue;
+
+                String data = this.replicationReader.convertMessageToJSON(message);
+
+                FlowFile flowFile = null;
+                flowFile = session.create();
+
+                flowFile = session.write(flowFile, new OutputStreamCallback() {
+                    @Override
+                    public void process(final OutputStream out) throws IOException {
+                        out.write(data.getBytes(StandardCharsets.UTF_8));
+                    }
+                });
+
+                flowFile = session.putAttribute(flowFile, "cdc.type", message.get("type").toString());
+
+                // Some messagens don't have LSN (e.g. Relation).
+                if (message.containsKey("lsn")) // Relation messagens don't have LSN.
+                    flowFile = session.putAttribute(flowFile, "cdc.lsn", message.get("lsn").toString());
+
+                flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), MIME_TYPE_DEFAULT);
+
+                listFlowFiles.add(flowFile);
+            }
+
+            session.transfer(listFlowFiles, REL_SUCCESS);
+
+            this.lastLSN = this.replicationReader.getLastReceiveLSN();
+
+            // Feedback is sent after the flowfiles transfer completes.
+            this.replicationReader.sendFeedback(this.lastLSN);
+
+            updateState(context.getStateManager());
+
+        } catch (SQLException | IOException e) {
+            context.yield();
+            this.closeQueryConnection();
+            this.closeReplicationConnection();
+            throw new ProcessException(e.getMessage(), e);
+        }
+    }
+
+    // Update last LSN received on processor state.
+    private void updateState(StateManager stateManager) throws IOException {
+        if (stateManager != null) {
+            Map<String, String> newStateMap = new HashMap<>(stateManager.getState(Scope.CLUSTER).toMap());
+
+            newStateMap.put(LAST_LSN_STATE_MAP_KEY, String.valueOf(this.lastLSN));
+            stateManager.setState(newStateMap, Scope.CLUSTER);
+        }
+    }
+
+    protected void stop(StateManager stateManager) throws CDCException {
+        try {
+            this.replicationReader = null;
+
+            if (lastLSN != null)
+                updateState(stateManager);
+
+            if (queryConnHolder != null)
+                queryConnHolder.close();
+
+            if (replicationConnHolder != null)
+                replicationConnHolder.close();
+
+        } catch (Exception e) {
+            throw new CDCException("Error closing CDC connections!", e);

Review comment:
       Sure! Thanks for the commit suggestion! I'll include that in the next commit.




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



[GitHub] [nifi] davyam commented on a change in pull request #5710: NIFI-4239 - Adding (REFACTORED) CaptureChangePostgreSQL processor to capture data changes (INSERT/UPDATE/DELETE) in PostgreSQL tables via Logical Replication

Posted by GitBox <gi...@apache.org>.
davyam commented on a change in pull request #5710:
URL: https://github.com/apache/nifi/pull/5710#discussion_r795088507



##########
File path: nifi-nar-bundles/nifi-cdc/nifi-cdc-postgresql-bundle/nifi-cdc-postgresql-processors/src/test/java/org/apache/nifi/cdc/postgresql/processors/ITCaptureChangePostgreSQL.java
##########
@@ -0,0 +1,1113 @@
+/*
+ * 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.
+ */
+package org.apache.nifi.cdc.postgresql.processors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.provenance.ProvenanceEventType;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runners.MethodSorters;
+import org.postgresql.replication.LogSequenceNumber;
+import org.postgresql.util.PSQLException;
+
+/**
+ * Integration Test for PostgreSQL CDC Processor.
+ *
+ * The PostgreSQL Cluster should be configured to enable logical replication:
+ *
+ * - Property listen_addresses should be set to the hostname to listen to;
+ *
+ * - Property max_wal_senders should be at least equal to the number of
+ * replication consumers;
+ *
+ * - Property wal_keep_size (or wal_keep_segments for PostgreSQL versions
+ * before 13) specifies the minimum size of past WAL segments kept in the WAL
+ * dir;
+ *
+ * - Property wal_level for logical replication should be equal to
+ * logical;
+ *
+ * - Property max_replication_slots should be greater than zero for logical
+ * replication.
+ *
+ * For example:
+ *
+ * <code>
+ * [postgresql.conf]
+ *
+ * listen_addresses = '*'
+ * max_wal_senders = 4
+ * wal_keep_size = 4
+ * wal_level = logical
+ * max_replication_slots = 4
+ * </code>
+ *
+ * For the Integration Tests it will be necessary to create specific users and
+ * database:
+ *
+ * <code>
+ * $ psql -h localhost -p 5432 -U postgres
+ *
+ * CREATE USER nifi SUPERUSER PASSWORD 'Change1t!';
+ * CREATE USER intern PASSWORD 'justForT3sts!';
+ *
+ * CREATE DATABASE db_test;
+ * </code>
+ *
+ * At last, authorize query and replication connections for user(s) and host(s).
+ * For example:
+ *
+ * <code>
+ * [pg_hba.conf]
+ *
+ * host db_test nifi 192.168.56.0/24 md5
+ * host db_test intern 192.168.56.0/24 md5
+ *
+ * host replication nifi 192.168.56.0/24 md5
+ * host replication intern 192.168.56.0/24 md5
+ * </code>
+ *
+ * These configurations requires the restart of PostgreSQL service.
+ */
+
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class ITCaptureChangePostgreSQL {
+
+        // Change it!
+        private static final String DRIVER_LOCATION = "/Users/davy.machado/Documents/SOFTWARES/LIBS/postgresql-42.3.1.jar";

Review comment:
       OK, I'll change the Integration Test to read the value of properties from environment variable.




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