Skip to content

Commit

Permalink
[FLINK-8014] [table] Add Kafka010JsonTableSink.
Browse files Browse the repository at this point in the history
- Refactor KafkaTableSink tests.
  • Loading branch information
fhueske committed Nov 16, 2017
1 parent fc3eebd commit 50fba9a
Show file tree
Hide file tree
Showing 11 changed files with 269 additions and 76 deletions.
@@ -0,0 +1,73 @@
/*
* 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.kafka;

import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
import org.apache.flink.types.Row;

import java.util.Properties;

/**
* Kafka 0.10 {@link KafkaTableSink} that serializes data in JSON format.
*/
public class Kafka010JsonTableSink extends KafkaJsonTableSink {

/**
* Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.10
* topic with fixed partition assignment.
*
* <p>Each parallel TableSink instance will write its rows to a single Kafka partition.</p>
* <ul>
* <li>If the number of Kafka partitions is less than the number of sink instances, different
* sink instances will write to the same partition.</li>
* <li>If the number of Kafka partitions is higher than the number of sink instance, some
* Kafka partitions won't receive data.</li>
* </ul>
*
* @param topic topic in Kafka to which table is written
* @param properties properties to connect to Kafka
*/
public Kafka010JsonTableSink(String topic, Properties properties) {
super(topic, properties, new FlinkFixedPartitioner<>());
}

/**
* Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.10
* topic with custom partition assignment.
*
* @param topic topic in Kafka to which table is written
* @param properties properties to connect to Kafka
* @param partitioner Kafka partitioner
*/
public Kafka010JsonTableSink(String topic, Properties properties, FlinkKafkaPartitioner<Row> partitioner) {
super(topic, properties, partitioner);
}

@Override
protected FlinkKafkaProducerBase<Row> createKafkaProducer(String topic, Properties properties, SerializationSchema<Row> serializationSchema, FlinkKafkaPartitioner<Row> partitioner) {
return new FlinkKafkaProducer010<>(topic, serializationSchema, properties, partitioner);
}

@Override
protected Kafka010JsonTableSink createCopy() {
return new Kafka010JsonTableSink(topic, properties, partitioner);
}
}
@@ -0,0 +1,53 @@
/*
* 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.kafka;

import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
import org.apache.flink.types.Row;

import java.util.Properties;

/**
* Tests for the {@link Kafka010JsonTableSink}.
*/
public class Kafka010JsonTableSinkTest extends KafkaTableSinkTestBase {

@Override
protected KafkaTableSink createTableSink(
String topic,
Properties properties,
FlinkKafkaPartitioner<Row> partitioner) {

return new Kafka010JsonTableSink(topic, properties, partitioner);
}

@Override
protected Class<? extends SerializationSchema<Row>> getSerializationSchemaClass() {
return JsonRowSerializationSchema.class;
}

@Override
protected Class<? extends FlinkKafkaProducerBase> getProducerClass() {
return FlinkKafkaProducer010.class;
}

}

Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.streaming.connectors.kafka; package org.apache.flink.streaming.connectors.kafka;


import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
Expand All @@ -32,7 +33,27 @@
public class Kafka08JsonTableSink extends KafkaJsonTableSink { public class Kafka08JsonTableSink extends KafkaJsonTableSink {


/** /**
* Creates {@link KafkaTableSink} for Kafka 0.8. * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.8
* topic with fixed partition assignment.
*
* <p>Each parallel TableSink instance will write its rows to a single Kafka partition.</p>
* <ul>
* <li>If the number of Kafka partitions is less than the number of sink instances, different
* sink instances will write to the same partition.</li>
* <li>If the number of Kafka partitions is higher than the number of sink instance, some
* Kafka partitions won't receive data.</li>
* </ul>
*
* @param topic topic in Kafka to which table is written
* @param properties properties to connect to Kafka
*/
public Kafka08JsonTableSink(String topic, Properties properties) {
super(topic, properties, new FlinkFixedPartitioner<>());
}

/**
* Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.8
* topic with custom partition assignment.
* *
* @param topic topic in Kafka to which table is written * @param topic topic in Kafka to which table is written
* @param properties properties to connect to Kafka * @param properties properties to connect to Kafka
Expand All @@ -43,7 +64,8 @@ public Kafka08JsonTableSink(String topic, Properties properties, FlinkKafkaParti
} }


/** /**
* Creates {@link KafkaTableSink} for Kafka 0.8. * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.8
* topic with custom partition assignment.
* *
* @param topic topic in Kafka to which table is written * @param topic topic in Kafka to which table is written
* @param properties properties to connect to Kafka * @param properties properties to connect to Kafka
Expand Down
Expand Up @@ -34,26 +34,19 @@ public class Kafka08JsonTableSinkTest extends KafkaTableSinkTestBase {
protected KafkaTableSink createTableSink( protected KafkaTableSink createTableSink(
String topic, String topic,
Properties properties, Properties properties,
FlinkKafkaPartitioner<Row> partitioner, FlinkKafkaPartitioner<Row> partitioner) {
final FlinkKafkaProducerBase<Row> kafkaProducer) {

return new Kafka08JsonTableSink(topic, properties, partitioner);
return new Kafka08JsonTableSink(topic, properties, partitioner) { }
@Override
protected FlinkKafkaProducerBase<Row> createKafkaProducer( @Override
String topic, protected Class<? extends SerializationSchema<Row>> getSerializationSchemaClass() {
Properties properties, return JsonRowSerializationSchema.class;
SerializationSchema<Row> serializationSchema,
FlinkKafkaPartitioner<Row> partitioner) {

return kafkaProducer;
}
};
} }


@Override @Override
@SuppressWarnings("unchecked") protected Class<? extends FlinkKafkaProducerBase> getProducerClass() {
protected SerializationSchema<Row> getSerializationSchema() { return FlinkKafkaProducer08.class;
return new JsonRowSerializationSchema(FIELD_NAMES);
} }
} }


Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.streaming.connectors.kafka; package org.apache.flink.streaming.connectors.kafka;


import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaDelegatePartitioner;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
Expand All @@ -32,7 +33,27 @@
public class Kafka09JsonTableSink extends KafkaJsonTableSink { public class Kafka09JsonTableSink extends KafkaJsonTableSink {


/** /**
* Creates {@link KafkaTableSink} for Kafka 0.9 . * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.9
* topic with fixed partition assignment.
*
* <p>Each parallel TableSink instance will write its rows to a single Kafka partition.</p>
* <ul>
* <li>If the number of Kafka partitions is less than the number of sink instances, different
* sink instances will write to the same partition.</li>
* <li>If the number of Kafka partitions is higher than the number of sink instance, some
* Kafka partitions won't receive data.</li>
* </ul>
*
* @param topic topic in Kafka to which table is written
* @param properties properties to connect to Kafka
*/
public Kafka09JsonTableSink(String topic, Properties properties) {
super(topic, properties, new FlinkFixedPartitioner<>());
}

/**
* Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.9
* topic with custom partition assignment.
* *
* @param topic topic in Kafka to which table is written * @param topic topic in Kafka to which table is written
* @param properties properties to connect to Kafka * @param properties properties to connect to Kafka
Expand All @@ -43,7 +64,8 @@ public Kafka09JsonTableSink(String topic, Properties properties, FlinkKafkaParti
} }


/** /**
* Creates {@link KafkaTableSink} for Kafka 0.9 . * Creates {@link KafkaTableSink} to write table rows as JSON-encoded records to a Kafka 0.9
* topic with custom partition assignment.
* *
* @param topic topic in Kafka to which table is written * @param topic topic in Kafka to which table is written
* @param properties properties to connect to Kafka * @param properties properties to connect to Kafka
Expand Down
Expand Up @@ -34,26 +34,19 @@ public class Kafka09JsonTableSinkTest extends KafkaTableSinkTestBase {
protected KafkaTableSink createTableSink( protected KafkaTableSink createTableSink(
String topic, String topic,
Properties properties, Properties properties,
FlinkKafkaPartitioner<Row> partitioner, FlinkKafkaPartitioner<Row> partitioner) {
final FlinkKafkaProducerBase<Row> kafkaProducer) {

return new Kafka09JsonTableSink(topic, properties, partitioner);
return new Kafka09JsonTableSink(topic, properties, partitioner) { }
@Override
protected FlinkKafkaProducerBase<Row> createKafkaProducer( @Override
String topic, protected Class<? extends SerializationSchema<Row>> getSerializationSchemaClass() {
Properties properties, return JsonRowSerializationSchema.class;
SerializationSchema<Row> serializationSchema,
FlinkKafkaPartitioner<Row> partitioner) {

return kafkaProducer;
}
};
} }


@Override @Override
@SuppressWarnings("unchecked") protected Class<? extends FlinkKafkaProducerBase> getProducerClass() {
protected SerializationSchema<Row> getSerializationSchema() { return FlinkKafkaProducer09.class;
return new JsonRowSerializationSchema(FIELD_NAMES);
} }
} }


Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.streaming.connectors.kafka; package org.apache.flink.streaming.connectors.kafka;


import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema;
import org.apache.flink.types.Row; import org.apache.flink.types.Row;
Expand All @@ -42,7 +43,7 @@ public KafkaJsonTableSink(String topic, Properties properties, FlinkKafkaPartiti
} }


@Override @Override
protected SerializationSchema<Row> createSerializationSchema(String[] fieldNames) { protected SerializationSchema<Row> createSerializationSchema(RowTypeInfo rowSchema) {
return new JsonRowSerializationSchema(fieldNames); return new JsonRowSerializationSchema(rowSchema);
} }
} }
Expand Up @@ -77,10 +77,10 @@ protected abstract FlinkKafkaProducerBase<Row> createKafkaProducer(
/** /**
* Create serialization schema for converting table rows into bytes. * Create serialization schema for converting table rows into bytes.
* *
* @param fieldNames Field names in table rows. * @param rowSchema the schema of the row to serialize.
* @return Instance of serialization schema * @return Instance of serialization schema
*/ */
protected abstract SerializationSchema<Row> createSerializationSchema(String[] fieldNames); protected abstract SerializationSchema<Row> createSerializationSchema(RowTypeInfo rowSchema);


/** /**
* Create a deep copy of this sink. * Create a deep copy of this sink.
Expand All @@ -92,6 +92,8 @@ protected abstract FlinkKafkaProducerBase<Row> createKafkaProducer(
@Override @Override
public void emitDataStream(DataStream<Row> dataStream) { public void emitDataStream(DataStream<Row> dataStream) {
FlinkKafkaProducerBase<Row> kafkaProducer = createKafkaProducer(topic, properties, serializationSchema, partitioner); FlinkKafkaProducerBase<Row> kafkaProducer = createKafkaProducer(topic, properties, serializationSchema, partitioner);
// always enable flush on checkpoint to achieve at-least-once if query runs with checkpointing enabled.
kafkaProducer.setFlushOnCheckpoint(true);
dataStream.addSink(kafkaProducer); dataStream.addSink(kafkaProducer);
} }


Expand All @@ -116,7 +118,9 @@ public KafkaTableSink configure(String[] fieldNames, TypeInformation<?>[] fieldT
copy.fieldTypes = Preconditions.checkNotNull(fieldTypes, "fieldTypes"); copy.fieldTypes = Preconditions.checkNotNull(fieldTypes, "fieldTypes");
Preconditions.checkArgument(fieldNames.length == fieldTypes.length, Preconditions.checkArgument(fieldNames.length == fieldTypes.length,
"Number of provided field names and types does not match."); "Number of provided field names and types does not match.");
copy.serializationSchema = createSerializationSchema(fieldNames);
RowTypeInfo rowSchema = new RowTypeInfo(fieldTypes, fieldNames);
copy.serializationSchema = createSerializationSchema(rowSchema);


return copy; return copy;
} }
Expand Down
Expand Up @@ -18,6 +18,9 @@
package org.apache.flink.streaming.util.serialization; package org.apache.flink.streaming.util.serialization;


import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.api.common.serialization.SerializationSchema;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.CompositeType;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
import org.apache.flink.types.Row; import org.apache.flink.types.Row;
import org.apache.flink.util.Preconditions; import org.apache.flink.util.Preconditions;


Expand All @@ -43,10 +46,23 @@ public class JsonRowSerializationSchema implements SerializationSchema<Row> {
/** /**
* Creates a JSON serialization schema for the given fields and types. * Creates a JSON serialization schema for the given fields and types.
* *
* @param fieldNames Names of JSON fields to parse. * @param rowSchema The schema of the rows to encode.
*/ */
public JsonRowSerializationSchema(String[] fieldNames) { public JsonRowSerializationSchema(RowTypeInfo rowSchema) {
this.fieldNames = Preconditions.checkNotNull(fieldNames);
Preconditions.checkNotNull(rowSchema);
String[] fieldNames = rowSchema.getFieldNames();
TypeInformation[] fieldTypes = rowSchema.getFieldTypes();

// check that no field is composite
for (int i = 0; i < fieldTypes.length; i++) {
if (fieldTypes[i] instanceof CompositeType) {
throw new IllegalArgumentException("JsonRowSerializationSchema cannot encode rows with nested schema, " +
"but field '" + fieldNames[i] + "' is nested: " + fieldTypes[i].toString());
}
}

this.fieldNames = fieldNames;
} }


@Override @Override
Expand Down

0 comments on commit 50fba9a

Please sign in to comment.