blob: c1301dade053b18bb47484bf3f1ae047b9d86ded [file] [log] [blame]
/*
* 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.RichOutputFormat;
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.streaming.connectors.kudu.serde.KuduSerialization;
import org.apache.flink.util.Preconditions;
import org.apache.kudu.client.SessionConfiguration.FlushMode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
public class KuduOutputFormat<OUT> extends RichOutputFormat<OUT> {
private static final long serialVersionUID = 1L;
private static final Logger LOG = LoggerFactory.getLogger(KuduOutputFormat.class);
private String kuduMasters;
private KuduTableInfo tableInfo;
private KuduConnector.Consistency consistency;
private KuduConnector.WriteMode writeMode;
private KuduSerialization<OUT> serializer;
private transient KuduConnector connector;
public KuduOutputFormat(String kuduMasters, KuduTableInfo tableInfo, KuduSerialization<OUT> serializer) {
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;
this.serializer = serializer.withSchema(tableInfo.getSchema());
}
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) {
}
@Override
public void open(int taskNumber, int numTasks) throws IOException {
if (connector != null) return;
connector = new KuduConnector(kuduMasters, tableInfo, consistency, writeMode,FlushMode.AUTO_FLUSH_SYNC);
serializer = serializer.withSchema(tableInfo.getSchema());
}
@Override
public void writeRecord(OUT row) throws IOException {
boolean response;
try {
KuduRow kuduRow = serializer.serialize(row);
response = connector.writeRow(kuduRow);
} catch (Exception e) {
throw new IOException(e.getLocalizedMessage(), e);
}
if(!response) {
throw new IOException("error with some transaction");
}
}
@Override
public void close() throws IOException {
if (this.connector == null) return;
try {
this.connector.close();
} catch (Exception e) {
throw new IOException(e.getLocalizedMessage(), e);
}
}
}