flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-2740) Create data consumer for Apache NiFi
Date Thu, 01 Oct 2015 10:28:26 GMT

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

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_r40899137
  
    --- Diff: flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-nifi/src/main/java/org/apache/flink/streaming/connectors/nifi/NiFiSink.java
---
    @@ -0,0 +1,74 @@
    +/**
    + * 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.sink.RichSinkFunction;
    +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.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +/**
    + * A sink that delivers data to Apache NiFi using the NiFi Site-to-Site client. The sink
requires
    + * a NiFiDataPacketBuilder which can create instances of NiFiDataPacket from the incoming
data.
    + */
    +public class NiFiSink<T> extends RichSinkFunction<T> {
    +
    +	private static final Logger LOG = LoggerFactory.getLogger(NiFiSink.class);
    +
    +	private SiteToSiteClient client;
    +	private SiteToSiteClientConfig clientConfig;
    +	private NiFiDataPacketBuilder<T> builder;
    +
    +	public NiFiSink(SiteToSiteClientConfig clientConfig, NiFiDataPacketBuilder<T>
builder) {
    +		this.clientConfig = clientConfig;
    +		this.builder = builder;
    +	}
    +
    +	@Override
    +	public void open(Configuration parameters) throws Exception {
    +		super.open(parameters);
    +		this.client = new SiteToSiteClient.Builder().fromConfig(clientConfig).build();
    +	}
    +
    +	@Override
    +	public void invoke(T value) throws Exception {
    +		final NiFiDataPacket niFiDataPacket = builder.createNiFiDataPacket(value, getRuntimeContext());
    +
    +		final Transaction transaction = client.createTransaction(TransferDirection.SEND);
    +		transaction.send(niFiDataPacket.getContent(), niFiDataPacket.getAttributes());
    +		transaction.confirm();
    +		transaction.complete();
    +	}
    +
    +	@Override
    +	public void close() throws Exception {
    +		super.close();
    +		try {
    +			client.close();
    +		} catch (final IOException ioe) {
    --- End diff --
    
    I think there is no need to catch this exception here. Flink will also log errors occurring
during the close() call.



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

Mime
View raw message