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

ASF GitHub Bot commented on FLINK-2740:
---------------------------------------

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

    https://github.com/apache/flink/pull/1198#discussion_r40899297
  
    --- Diff: 
flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSource.java
 ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.flink.streaming.connectors.nifi;
    +
    +import org.apache.flink.configuration.Configuration;
    +import 
org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
    +import org.apache.nifi.remote.Transaction;
    +import org.apache.nifi.remote.TransferDirection;
    +import org.apache.nifi.remote.client.SiteToSiteClient;
    +import org.apache.nifi.remote.client.SiteToSiteClientConfig;
    +import org.apache.nifi.remote.protocol.DataPacket;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Map;
    +
    +/**
    + * A source that pulls data from Apache NiFi using the NiFi Site-to-Site 
client. This source
    + * produces NiFiDataPackets which encapsulate the content and attributes 
of a NiFi FlowFile.
    + */
    +public class NiFiSource extends RichParallelSourceFunction<NiFiDataPacket> 
{
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(NiFiSource.class);
    +
    +   private SiteToSiteClient client;
    +   private SiteToSiteClientConfig clientConfig;
    +   private transient volatile boolean running;
    +
    +   public NiFiSource(SiteToSiteClientConfig clientConfig) {
    +           this.clientConfig = clientConfig;
    +   }
    +
    +   @Override
    +   public void open(Configuration parameters) throws Exception {
    +           super.open(parameters);
    +           client = new 
SiteToSiteClient.Builder().fromConfig(clientConfig).build();
    +           running = true;
    +   }
    +
    +   @Override
    +   public void run(SourceContext<NiFiDataPacket> ctx) throws Exception {
    +           while (running) {
    +                   final Transaction transaction = 
client.createTransaction(TransferDirection.RECEIVE);
    +
    +                   DataPacket dataPacket = transaction.receive();
    +                   if (dataPacket == null) {
    +                           transaction.confirm();
    +                           transaction.complete();
    +
    +                           // no data available. Wait a bit and try again
    +                           try {
    +                                   Thread.sleep(1000L);
    +                           } catch (InterruptedException e) {
    +
    +                           }
    +                           continue;
    +                   }
    +
    +                   final List<NiFiDataPacket> niFiDataPackets = new 
ArrayList<>();
    +                   do {
    +                           // Read the data into a byte array and wrap it 
along with the attributes
    +                           // into a NiFiDataPacket.
    +                           final InputStream inStream = 
dataPacket.getData();
    +                           final byte[] data = new byte[(int) 
dataPacket.getSize()];
    +                           StreamUtils.fillBuffer(inStream, data);
    +
    +                           final Map<String, String> attributes = 
dataPacket.getAttributes();
    +
    +                           niFiDataPackets.add(new 
StandardNiFiDataPacket(data, attributes));
    +                           dataPacket = transaction.receive();
    +                   } while (dataPacket != null);
    +
    +                   // Confirm transaction to verify the data
    +                   transaction.confirm();
    +
    +                   for (NiFiDataPacket dp : niFiDataPackets) {
    +                           ctx.collect(dp);
    +                   }
    +
    +                   transaction.complete();
    +           }
    +   }
    +
    +   @Override
    +   public void cancel() {
    +           running = false;
    +   }
    +
    +   @Override
    +   public void close() throws Exception {
    +           super.close();
    +           try {
    +                   client.close();
    +           } catch (final IOException ioe) {
    +                   LOG.error("Unable to close SiteToSiteClient: " + 
ioe.getMessage());
    --- End diff --
    
    Same here. Flink can handle the Exception


> Create data consumer for Apache NiFi
> ------------------------------------
>
>                 Key: FLINK-2740
>                 URL: https://issues.apache.org/jira/browse/FLINK-2740
>             Project: Flink
>          Issue Type: New Feature
>          Components: Streaming Connectors
>            Reporter: Kostas Tzoumas
>            Assignee: Joseph Witt
>
> Create a connector to Apache NiFi to create Flink DataStreams from NiFi flows



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to