You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/03/30 03:25:42 UTC

[jira] [Commented] (NIFI-3413) Implement a CaptureChangeMySQL processor

    [ https://issues.apache.org/jira/browse/NIFI-3413?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15948348#comment-15948348 ] 

ASF GitHub Bot commented on NIFI-3413:
--------------------------------------

Github user ijokarumawak commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/1618#discussion_r108827773
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CaptureChangeMySQL.java ---
    @@ -0,0 +1,928 @@
    +/*
    + * 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.processors.standard;
    +
    +import com.github.shyiko.mysql.binlog.BinaryLogClient;
    +import com.github.shyiko.mysql.binlog.event.Event;
    +import com.github.shyiko.mysql.binlog.event.EventHeaderV4;
    +import com.github.shyiko.mysql.binlog.event.EventType;
    +import com.github.shyiko.mysql.binlog.event.QueryEventData;
    +import com.github.shyiko.mysql.binlog.event.RotateEventData;
    +import com.github.shyiko.mysql.binlog.event.TableMapEventData;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +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.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +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.distributed.cache.client.Deserializer;
    +import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
    +import org.apache.nifi.distributed.cache.client.Serializer;
    +import org.apache.nifi.logging.ComponentLog;
    +import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.ProcessSessionFactory;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.standard.db.CDCException;
    +import org.apache.nifi.processors.standard.db.event.ColumnDefinition;
    +import org.apache.nifi.processors.standard.db.event.RowEventException;
    +import org.apache.nifi.processors.standard.db.event.TableInfo;
    +import org.apache.nifi.processors.standard.db.event.TableInfoCacheKey;
    +import org.apache.nifi.processors.standard.db.event.io.EventWriter;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.BeginTransactionEventInfo;
    +import org.apache.nifi.processors.standard.db.impl.mysql.RawBinlogEvent;
    +import org.apache.nifi.processors.standard.db.impl.mysql.BinlogEventListener;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.BinlogEventInfo;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.CommitTransactionEventInfo;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.DeleteRowsEventInfo;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.SchemaChangeEventInfo;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.UpdateRowsEventInfo;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.InsertRowsEventInfo;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.io.BeginTransactionEventWriter;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.io.CommitTransactionEventWriter;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.io.DeleteRowsWriter;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.io.InsertRowsWriter;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.io.SchemaChangeEventWriter;
    +import org.apache.nifi.processors.standard.db.impl.mysql.event.io.UpdateRowsWriter;
    +import org.apache.nifi.reporting.InitializationException;
    +import org.apache.nifi.util.StringUtils;
    +import org.apache.nifi.util.file.classloader.ClassLoaderUtils;
    +
    +import java.io.IOException;
    +import java.net.ConnectException;
    +import java.net.InetSocketAddress;
    +import java.net.MalformedURLException;
    +import java.sql.Connection;
    +import java.sql.Driver;
    +import java.sql.DriverManager;
    +import java.sql.DriverPropertyInfo;
    +import java.sql.ResultSet;
    +import java.sql.ResultSetMetaData;
    +import java.sql.SQLException;
    +import java.sql.SQLFeatureNotSupportedException;
    +import java.sql.Statement;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +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.LinkedBlockingQueue;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicLong;
    +import java.util.logging.Logger;
    +import java.util.regex.Pattern;
    +
    +/**
    + * A processor to retrieve Change Data Capture (CDC) events and send them as flow files.
    + */
    +@TriggerSerially
    +@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
    +@Tags({"sql", "jdbc", "cdc", "mysql"})
    +@CapabilityDescription("Retrieves Change Data Capture (CDC) events from a MySQL database. CDC Events include INSERT, UPDATE, DELETE operations. Events "
    +        + "are output as individual flow files ordered by the time at which the operation occurred.")
    +@Stateful(scopes = Scope.LOCAL, description = "Information such as a 'pointer' to the current CDC event in the database is stored by this processor, such "
    +        + "that it can continue from the same location if restarted.")
    +@DynamicProperties({
    +        @DynamicProperty(
    +                name = "init.binlog.filename",
    +                value = "The binlog filename to start from",
    +                description = "Specifies the name of the binlog file from which to begin retrieving CDC records."
    +        ),
    +        @DynamicProperty(
    +                name = "init.binlog.position",
    +                value = "The offset into the initial binlog file to start from",
    +                description = "Specifies the offset into the initial binlog file from which to begin retrieving CDC records."
    +        )
    +})
    +@WritesAttributes({
    +        @WritesAttribute(attribute = "cdc.sequence.id", description = "A sequence identifier (i.e. strictly increasing integer value) specifying the order "
    +                + "of the CDC event flow file relative to the other event flow file(s)."),
    +        @WritesAttribute(attribute = "cdc.event.type", description = "A string indicating the type of CDC event that occurred, including (but not limited to) "
    +                + "'begin', 'write', 'update', 'delete', 'schema_change' and 'commit'."),
    +        @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 CaptureChangeMySQL extends AbstractSessionFactoryProcessor {
    +
    +    // Relationships
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Successfully created FlowFile from SQL query result set.")
    +            .build();
    +
    +    protected static Set<Relationship> relationships;
    +
    +    // Properties
    +    public static final PropertyDescriptor DATABASE_NAME_PATTERN = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-db-name-pattern")
    +            .displayName("Database/Schema Name Pattern")
    +            .description("A regular expression (regex) for matching databases or schemas (depending on your RDBMS' terminology) against the list of CDC events. The regex must match "
    +                    + "the schema name as it is stored in the database. If the property is not set, the schema name will not be used to filter the CDC events.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor TABLE_NAME_PATTERN = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-name-pattern")
    +            .displayName("Table Name Pattern")
    +            .description("A regular expression (regex) for matching CDC events affecting matching tables. The regex must match the table name as it is stored in the database. "
    +                    + "If the property is not set, no events will be filtered based on table name.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor CONNECT_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-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. Max time less than 1 second will be equal to zero.")
    +            .defaultValue("0 seconds")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HOSTS = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-hosts")
    +            .displayName("MySQL Hosts")
    +            .description("A list of hostname/port entries corresponding to nodes in a MySQL cluster. The entries should be comma separated "
    +                    + "using a colon such as host1:port,host2:port,....  For example mysql.myhost.com:3306. This processor will attempt to connect to "
    +                    + "the hosts in the list in order. If one node goes down and failover is enabled for the cluster, then the processor will connect "
    +                    + "to the active node (assuming its host entry is specified in this property.  The default port for MySQL connections is 3306.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(StandardValidators.HOSTNAME_PORT_LIST_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor DRIVER_NAME = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-driver-class")
    +            .displayName("MySQL Driver Class Name")
    +            .description("The class name of the MySQL database driver class")
    +            .defaultValue("com.mysql.jdbc.Driver")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor DRIVER_LOCATION = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-driver-locations")
    +            .displayName("MySQL Driver Location(s)")
    +            .description("Comma-separated list of files/folders and/or URLs containing the MySQL driver JAR and its dependencies (if any). "
    +                    + "For example '/var/tmp/mysql-connector-java-5.1.38-bin.jar'")
    +            .defaultValue(null)
    +            .required(false)
    +            .addValidator(StandardValidators.createListValidator(true, true, StandardValidators.createURLorFileValidator()))
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-username")
    +            .displayName("Username")
    +            .description("Username to access the MySQL cluster")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-password")
    +            .displayName("Password")
    +            .description("Password to access the MySQL cluster")
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor DIST_CACHE_CLIENT = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-dist-map-cache-client")
    +            .displayName("Distributed Map Cache Client")
    +            .description("Identifies a Distributed Map Cache Client controller service to be used for keeping information about the various tables, columns, etc. "
    +                    + "needed by the processor. If a client is not specified, the generated events will not include column type or name information.")
    +            .identifiesControllerService(DistributedMapCacheClient.class)
    +            .required(false)
    +            .build();
    +
    +    public static final PropertyDescriptor RETRIEVE_ALL_RECORDS = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-retrieve-all-records")
    +            .displayName("Retrieve All Records")
    +            .description("Specifies whether to get all available CDC events, regardless of the current binlog filename and/or position. If this property is set to true, "
    +                    + "any init.binlog.filename and init.binlog.position dynamic properties are ignored. NOTE: If binlog filename and position values are present in the processor's "
    +                    + "State Map, this property's value is ignored. To reset the behavior, clear the processor state (refer to the State Management section of the processor's documentation.")
    +            .required(true)
    +            .allowableValues("true", "false")
    +            .defaultValue("true")
    +            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor INIT_SEQUENCE_ID = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-init-seq-id")
    +            .displayName("Initial Sequence ID")
    +            .description("Specifies an initial sequence identifier to use if this processor's State does not have a current "
    +                    + "sequence identifier. If a sequence identifier is present in the processor's State, this property is ignored. Sequence identifiers are "
    +                    + "monotonically increasing integers that record the order of flow files generated by the processor. They can be used with the EnforceOrder "
    +                    + "processor to guarantee ordered delivery of CDC events.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor INIT_BINLOG_FILENAME = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-init-binlog-filename")
    +            .displayName("Initial Binlog Filename")
    +            .description("Specifies an initial binlog filename to use if this processor's State does not have a current binlog filename. If a filename is present "
    +                    + "in the processor's State, this property is ignored. This can be used along with Initial Binlog Position to \"skip ahead\" if previous events are not desired. "
    +                    + "Note that NiFi Expression Language is supported, but this property is evaluated when the processor is configured, so FlowFile attributes may not be used. Expression "
    +                    + "Language is supported to enable the use of the Variable Registry and/or environment properties.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor INIT_BINLOG_POSITION = new PropertyDescriptor.Builder()
    +            .name("capture-change-mysql-init-binlog-position")
    +            .displayName("Initial Binlog Position")
    +            .description("Specifies an initial offset into a binlog (specified by Initial Binlog Filename) to use if this processor's State does not have a current "
    +                    + "binlog filename. If a filename is present in the processor's State, this property is ignored. This can be used along with Initial Binlog Filename "
    +                    + "to \"skip ahead\" if previous events are not desired. Note that NiFi Expression Language is supported, but this property is evaluated when the "
    +                    + "processor is configured, so FlowFile attributes may not be used. Expression Language is supported to enable the use of the Variable Registry "
    +                    + "and/or environment properties.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    private static List<PropertyDescriptor> propDescriptors;
    +
    +    private volatile ProcessSession currentSession;
    +    private BinaryLogClient binlogClient;
    +    private BinlogEventListener eventListener;
    +
    +    private volatile LinkedBlockingQueue<RawBinlogEvent> queue = new LinkedBlockingQueue<>();
    +    private volatile String currentBinlogFile = null;
    +    private volatile long currentBinlogPosition = 4;
    +
    +    // The following variables save the value of the binlog filename and position (and sequence id) at the beginning of a transaction. Used for rollback
    +    private volatile String xactBinlogFile = null;
    +    private volatile long xactBinlogPosition = 4;
    +    private volatile long xactSequenceId = 0;
    +
    +    private volatile TableInfo currentTable = null;
    +    private volatile Pattern databaseNamePattern;
    +    private volatile Pattern tableNamePattern;
    +
    +    private volatile boolean inTransaction = false;
    +    private volatile boolean skipTable = false;
    +    private AtomicBoolean doStop = new AtomicBoolean(false);
    +
    +    private int currentHost = 0;
    +
    +    private AtomicLong currentSequenceId = new AtomicLong(0);
    +
    +    private volatile DistributedMapCacheClient cacheClient = null;
    +    private final Serializer<TableInfoCacheKey> cacheKeySerializer = new TableInfoCacheKey.Serializer();
    +    private final Serializer<TableInfo> cacheValueSerializer = new TableInfo.Serializer();
    +    private final Deserializer<TableInfo> cacheValueDeserializer = new TableInfo.Deserializer();
    +
    +    private Connection jdbcConnection = null;
    +
    +    private static final BeginTransactionEventWriter beginEventWriter = new BeginTransactionEventWriter();
    +    private static final CommitTransactionEventWriter commitEventWriter = new CommitTransactionEventWriter();
    +    private static final SchemaChangeEventWriter schemaChangeEventWriter = new SchemaChangeEventWriter();
    +    private static final InsertRowsWriter insertRowsWriter = new InsertRowsWriter();
    +    private static final DeleteRowsWriter deleteRowsWriter = new DeleteRowsWriter();
    +    private static final UpdateRowsWriter updateRowsWriter = new UpdateRowsWriter();
    +
    +    static {
    +
    +        final Set<Relationship> r = new HashSet<>();
    +        r.add(REL_SUCCESS);
    +        relationships = Collections.unmodifiableSet(r);
    +
    +        final List<PropertyDescriptor> pds = new ArrayList<>();
    +        pds.add(HOSTS);
    +        pds.add(DRIVER_NAME);
    +        pds.add(DRIVER_LOCATION);
    +        pds.add(USERNAME);
    +        pds.add(PASSWORD);
    +        pds.add(DATABASE_NAME_PATTERN);
    +        pds.add(TABLE_NAME_PATTERN);
    +        pds.add(CONNECT_TIMEOUT);
    +        pds.add(DIST_CACHE_CLIENT);
    +        pds.add(RETRIEVE_ALL_RECORDS);
    +        pds.add(INIT_SEQUENCE_ID);
    +        pds.add(INIT_BINLOG_FILENAME);
    +        pds.add(INIT_BINLOG_POSITION);
    +        propDescriptors = Collections.unmodifiableList(pds);
    +    }
    +
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return propDescriptors;
    +    }
    +
    +    @Override
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    --- End diff --
    
    `INIT_BINLOG_FILENAME` and `INIT_BINLOG_POSITION` are defined as supported properties. Why do we need this to be defined as dynamic also?


> Implement a CaptureChangeMySQL processor
> ----------------------------------------
>
>                 Key: NIFI-3413
>                 URL: https://issues.apache.org/jira/browse/NIFI-3413
>             Project: Apache NiFi
>          Issue Type: New Feature
>          Components: Extensions
>            Reporter: Matt Burgess
>            Assignee: Matt Burgess
>             Fix For: 1.2.0
>
>
> Database systems such as MySQL, Oracle, and SQL Server allow access to their transactional logs and such, in order for external clients to have a "change data capture" (CDC) capability. As an initial effort, I propose a CaptureChangeMySQL processor to enable this in NiFi. This would incorporate any APIs necessary for follow-on Jira cases to implement CDC processors for databases such as Oracle, SQL Server, PostgreSQL, etc.
> The processor would include properties needed for database connectivity (unless using a DBCPConnectionPool would suffice), as well as any to configure third-party clients (mysql-binlog-connector, e.g.). It would also need to keep a "sequence ID" such that an EnforceOrder processor (NIFI-3414) for example could guarantee the order of CDC events for use cases such as replication. It will likely need State Management for that, and may need other facilities such as a DistributedMapCache in order to keep information (column names and types, e.g.) that enrich the raw CDC events.
> The processor would accept no incoming connections (it is a "get" or source processor), would be intended to run on the primary node only as a single threaded processor, and would generate a flow file for each operation (INSERT, UPDATE, DELETE, e,g,) in one or some number of formats (JSON, e.g.).



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)