Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
1 parent
9045ce4
commit 54df0a4
Showing
14 changed files
with
944 additions
and
3 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
31 changes: 31 additions & 0 deletions
31
presto-product-tests/conf/presto/etc/catalog/kafka/all_datatypes_avro.json
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
{ | ||
"tableName": "all_datatypes_avro", | ||
"schemaName": "product_tests", | ||
"topicName": "all_datatypes_avro", | ||
"message": { | ||
"dataFormat": "avro", | ||
"dataSchema": "/docker/volumes/conf/presto/etc/catalog/kafka/all_datatypes_avro_schema.avsc", | ||
"fields": [ | ||
{ | ||
"name": "c_varchar", | ||
"type": "VARCHAR", | ||
"mapping": "a_varchar" | ||
}, | ||
{ | ||
"name": "c_bigint", | ||
"type": "BIGINT", | ||
"mapping": "a_bigint" | ||
}, | ||
{ | ||
"name": "c_double", | ||
"type": "DOUBLE", | ||
"mapping": "a_double" | ||
}, | ||
{ | ||
"name": "c_boolean", | ||
"type": "BOOLEAN", | ||
"mapping": "a_boolean" | ||
} | ||
] | ||
} | ||
} |
29 changes: 29 additions & 0 deletions
29
presto-product-tests/conf/presto/etc/catalog/kafka/all_datatypes_avro_schema.avsc
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,29 @@ | ||
{ | ||
"type" : "record", | ||
"name" : "all_datatypes_avro", | ||
"namespace" : "com.facebook.presto.tests.kafka", | ||
"fields" : | ||
[ | ||
{ | ||
"name":"a_varchar", | ||
"type":["null", "string"], | ||
"default": "null" | ||
}, | ||
{ | ||
"name":"a_bigint", | ||
"type":["null", "long"], | ||
"default": null | ||
}, | ||
{ | ||
"name":"a_double", | ||
"type":["null", "double"], | ||
"default": null | ||
}, | ||
{ | ||
"name":"a_boolean", | ||
"type":["null", "boolean"], | ||
"default": null | ||
} | ||
], | ||
"doc:" : "A basic avro schema for product tests" | ||
} |
31 changes: 31 additions & 0 deletions
31
presto-product-tests/conf/presto/etc/catalog/kafka/all_null_avro.json
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
{ | ||
"tableName": "all_null_avro", | ||
"schemaName": "product_tests", | ||
"topicName": "all_null_avro", | ||
"message": { | ||
"dataFormat": "avro", | ||
"dataSchema": "/docker/volumes/conf/presto/etc/catalog/kafka/all_datatypes_avro_schema.avsc", | ||
"fields": [ | ||
{ | ||
"name": "c_varchar", | ||
"type": "VARCHAR", | ||
"mapping": "a_varchar" | ||
}, | ||
{ | ||
"name": "c_bigint", | ||
"type": "BIGINT", | ||
"mapping": "a_bigint" | ||
}, | ||
{ | ||
"name": "c_double", | ||
"type": "DOUBLE", | ||
"mapping": "a_double" | ||
}, | ||
{ | ||
"name": "c_boolean", | ||
"type": "BOOLEAN", | ||
"mapping": "a_boolean" | ||
} | ||
] | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
151 changes: 151 additions & 0 deletions
151
presto-product-tests/src/main/java/com/facebook/presto/tests/kafka/KafkaAvroSmokeTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,151 @@ | ||
/* | ||
* Licensed 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 com.facebook.presto.tests.kafka; | ||
|
||
import com.google.common.collect.ImmutableList; | ||
import com.google.common.collect.ImmutableMap; | ||
import io.prestodb.tempto.ProductTest; | ||
import io.prestodb.tempto.Requirement; | ||
import io.prestodb.tempto.RequirementsProvider; | ||
import io.prestodb.tempto.Requires; | ||
import io.prestodb.tempto.configuration.Configuration; | ||
import io.prestodb.tempto.fulfillment.table.kafka.KafkaMessage; | ||
import io.prestodb.tempto.fulfillment.table.kafka.KafkaTableDefinition; | ||
import io.prestodb.tempto.fulfillment.table.kafka.ListKafkaDataSource; | ||
import io.prestodb.tempto.query.QueryResult; | ||
import org.apache.avro.Schema; | ||
import org.apache.avro.file.DataFileWriter; | ||
import org.apache.avro.generic.GenericData; | ||
import org.apache.avro.generic.GenericDatumWriter; | ||
import org.apache.avro.generic.GenericRecord; | ||
import org.testng.annotations.Test; | ||
|
||
import java.io.ByteArrayOutputStream; | ||
import java.io.File; | ||
import java.io.IOException; | ||
import java.io.UncheckedIOException; | ||
import java.sql.SQLException; | ||
import java.util.Map; | ||
|
||
import static com.facebook.presto.tests.TestGroups.KAFKA; | ||
import static io.prestodb.tempto.assertions.QueryAssert.Row.row; | ||
import static io.prestodb.tempto.assertions.QueryAssert.assertThat; | ||
import static io.prestodb.tempto.fulfillment.table.TableRequirements.immutableTable; | ||
import static io.prestodb.tempto.fulfillment.table.kafka.KafkaMessageContentsBuilder.contentsBuilder; | ||
import static io.prestodb.tempto.query.QueryExecutor.query; | ||
import static java.lang.String.format; | ||
|
||
public class KafkaAvroSmokeTest | ||
extends ProductTest | ||
{ | ||
private static final String KAFKA_CATALOG = "kafka"; | ||
|
||
private static final String ALL_DATATYPES_AVRO_TABLE_NAME = "product_tests.all_datatypes_avro"; | ||
private static final String ALL_DATATYPES_AVRO_TOPIC_NAME = "all_datatypes_avro"; | ||
private static final String ALL_DATATYPE_SCHEMA_PATH = "/docker/volumes/conf/presto/etc/catalog/kafka/all_datatypes_avro_schema.avsc"; | ||
|
||
private static final String ALL_NULL_AVRO_TABLE_NAME = "product_tests.all_null_avro"; | ||
private static final String ALL_NULL_AVRO_TOPIC_NAME = "all_null_avro"; | ||
// kafka-connectors requires tables to be predefined in presto configuration | ||
// the requirements here will be used to verify that table actually exists and to | ||
// create topics and propagate them with data | ||
private static class AllDataTypesAvroTable | ||
implements RequirementsProvider | ||
{ | ||
@Override | ||
public Requirement getRequirements(Configuration configuration) | ||
{ | ||
ImmutableMap<String, Object> record = ImmutableMap.of( | ||
"a_varchar", "foobar", | ||
"a_bigint", 127L, | ||
"a_double", 234.567, | ||
"a_boolean", true); | ||
return createAvroTable(ALL_DATATYPE_SCHEMA_PATH, ALL_DATATYPES_AVRO_TABLE_NAME, ALL_DATATYPES_AVRO_TOPIC_NAME, record); | ||
} | ||
} | ||
|
||
private static Requirement createAvroTable(String schemaPath, String tableName, String topicName, ImmutableMap<String, Object> record) | ||
{ | ||
try { | ||
Schema schema = new Schema.Parser().parse(new File(schemaPath)); | ||
byte[] avroData = convertRecordToAvro(schema, record); | ||
|
||
return immutableTable(new KafkaTableDefinition( | ||
tableName, | ||
topicName, | ||
new ListKafkaDataSource(ImmutableList.of( | ||
new KafkaMessage( | ||
contentsBuilder() | ||
.appendBytes(avroData) | ||
.build()))), | ||
1, | ||
1)); | ||
} | ||
catch (IOException e) { | ||
throw new UncheckedIOException(e); | ||
} | ||
} | ||
|
||
private static byte[] convertRecordToAvro(Schema schema, Map<String, Object> values) | ||
{ | ||
ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); | ||
GenericData.Record record = new GenericData.Record(schema); | ||
values.forEach(record::put); | ||
try (DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(new GenericDatumWriter<>(schema))) { | ||
dataFileWriter.create(schema, outputStream); | ||
dataFileWriter.append(record); | ||
dataFileWriter.close(); | ||
} | ||
catch (IOException e) { | ||
throw new UncheckedIOException("Failed to convert to AVRO.", e); | ||
} | ||
return outputStream.toByteArray(); | ||
} | ||
|
||
@Test(groups = {KAFKA}) | ||
@Requires(AllDataTypesAvroTable.class) | ||
public void testSelectPrimitiveDataType() | ||
throws SQLException | ||
{ | ||
QueryResult queryResult = query(format("select * from %s.%s", KAFKA_CATALOG, ALL_DATATYPES_AVRO_TABLE_NAME)); | ||
assertThat(queryResult).containsOnly(row( | ||
"foobar", | ||
127, | ||
234.567, | ||
true)); | ||
} | ||
|
||
private static class NullDataAvroTable | ||
implements RequirementsProvider | ||
{ | ||
@Override | ||
public Requirement getRequirements(Configuration configuration) | ||
{ | ||
return createAvroTable(ALL_DATATYPE_SCHEMA_PATH, ALL_NULL_AVRO_TABLE_NAME, ALL_NULL_AVRO_TOPIC_NAME, ImmutableMap.of()); | ||
} | ||
} | ||
|
||
@Test(groups = {KAFKA}) | ||
@Requires(NullDataAvroTable.class) | ||
public void testNullType() | ||
throws SQLException | ||
{ | ||
QueryResult queryResult = query(format("select * from %s.%s", KAFKA_CATALOG, ALL_NULL_AVRO_TABLE_NAME)); | ||
assertThat(queryResult).containsOnly(row( | ||
null, | ||
null, | ||
null, | ||
null)); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.