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

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

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

    https://github.com/apache/nifi/pull/1618#discussion_r109704801
  
    --- Diff: 
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/CaptureChangeMySQL.java
 ---
    @@ -0,0 +1,1001 @@
    +/*
    + * 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.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.OnShutdown;
    +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.BinlogLifecycleListener;
    +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.CLUSTER, 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.")
    +@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 {
    +
    +    // Random invalid constant used as an indicator to not set the binlog 
position on the client (thereby using the latest available)
    +    private static final int DO_NOT_SET = -1000;
    +
    +    // 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")
    --- End diff --
    
    Hmm I used to have the default as 30 seconds, not sure where that code went 
:( Will make the change.


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

Reply via email to