[ https://issues.apache.org/jira/browse/BAHIR-99?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16570733#comment-16570733 ]
ASF GitHub Bot commented on BAHIR-99: ------------------------------------- Github user lresende commented on a diff in the pull request: https://github.com/apache/bahir-flink/pull/31#discussion_r208012315 --- Diff: flink-connector-kudu/src/main/java/org/apache/flink/streaming/connectors/kudu/KuduOutputFormat.java --- @@ -0,0 +1,110 @@ +/* + * 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.kudu; + +import org.apache.flink.api.common.io.OutputFormat; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.connectors.kudu.connector.KuduConnector; +import org.apache.flink.streaming.connectors.kudu.connector.KuduRow; +import org.apache.flink.streaming.connectors.kudu.connector.KuduTableInfo; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class KuduOutputFormat<OUT extends KuduRow> implements OutputFormat<OUT> { + + private static final Logger LOG = LoggerFactory.getLogger(KuduOutputFormat.class); + + private String kuduMasters; + private KuduTableInfo tableInfo; + private KuduConnector.Consistency consistency; + private KuduConnector.WriteMode writeMode; + + private transient KuduConnector tableContext; + + + public KuduOutputFormat(String kuduMasters, KuduTableInfo tableInfo) { + Preconditions.checkNotNull(kuduMasters,"kuduMasters could not be null"); + this.kuduMasters = kuduMasters; + + Preconditions.checkNotNull(tableInfo,"tableInfo could not be null"); + this.tableInfo = tableInfo; + this.consistency = KuduConnector.Consistency.STRONG; + this.writeMode = KuduConnector.WriteMode.UPSERT; + } + + public KuduOutputFormat<OUT> withEventualConsistency() { + this.consistency = KuduConnector.Consistency.EVENTUAL; + return this; + } + + public KuduOutputFormat<OUT> withStrongConsistency() { + this.consistency = KuduConnector.Consistency.STRONG; + return this; + } + + public KuduOutputFormat<OUT> withUpsertWriteMode() { + this.writeMode = KuduConnector.WriteMode.UPSERT; + return this; + } + + public KuduOutputFormat<OUT> withInsertWriteMode() { + this.writeMode = KuduConnector.WriteMode.INSERT; + return this; + } + + public KuduOutputFormat<OUT> withUpdateWriteMode() { + this.writeMode = KuduConnector.WriteMode.UPDATE; + return this; + } + + @Override + public void configure(Configuration parameters) { --- End diff -- Are we not supporting configuring the extension? > Kudu connector to read/write from/to Kudu > ----------------------------------------- > > Key: BAHIR-99 > URL: https://issues.apache.org/jira/browse/BAHIR-99 > Project: Bahir > Issue Type: New Feature > Components: Flink Streaming Connectors > Affects Versions: Flink-1.0 > Reporter: Rubén Casado > Assignee: Rubén Casado > Priority: Major > Fix For: Flink-Next > > > Java library to integrate Apache Kudu and Apache Flink. Main goal is to be > able to read/write data from/to Kudu using the DataSet and DataStream Flink's > APIs. > Data flows patterns: > Batch > - Kudu -> DataSet<RowSerializable> -> Kudu > - Kudu -> DataSet<RowSerializable> -> other source > - Other source -> DataSet<RowSerializable> -> other source > Stream > - Other source -> DataStream <RowSerializable> -> Kudu > Code is available in https://github.com/rubencasado/Flink-Kudu -- This message was sent by Atlassian JIRA (v7.6.3#76005)