diff --git a/docs/content/development/extensions-contrib/thrift.md b/docs/content/development/extensions-contrib/thrift.md new file mode 100644 index 00000000000..a25355b17aa --- /dev/null +++ b/docs/content/development/extensions-contrib/thrift.md @@ -0,0 +1,104 @@ +--- +layout: doc_page +--- + +# Thrift + +To use this extension, make sure to [include](../../operations/including-extensions.html) `druid-thrift-extensions`. + +This extension enables Druid to ingest thrift compact data online (`ByteBuffer`) and offline (SequenceFile of type `` or LzoThriftBlock File). + +You may want to use another version of thrift, change the dependency in pom and compile yourself. + +## Thrift Parser + + +| Field | Type | Description | Required | +| ----------- | ----------- | ---------------------------------------- | -------- | +| type | String | This should say `thrift` | yes | +| parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Should be a Json parseSpec. | yes | +| thriftJar | String | path of thrift jar, if not provided, it will try to find the thrift class in classpath. Thrift jar in batch ingestion should be uploaded to HDFS first and configure `jobProperties` with `"tmpjars":"/path/to/your/thrift.jar"` | no | +| thriftClass | String | classname of thrift | yes | + +- Realtime Ingestion (tranquility example) + +```json +{ + "dataSources": [{ + "spec": { + "dataSchema": { + "dataSource": "book", + "granularitySpec": { }, + "parser": { + "type": "thrift", + "thriftClass": "io.druid.data.input.thrift.Book", + "protocol": "compact", + "parseSpec": { + "format": "json", + ... + } + }, + "metricsSpec": [...] + }, + "tuningConfig": {...} + }, + "properties": {...} + }], + "properties": {...} +} +``` + +To use it with tranquility, + +```bash +bin/tranquility kafka \ + -configFile $jsonConfig \ + -Ddruid.extensions.directory=/path/to/extensions \ + -Ddruid.extensions.loadList='["druid-thrift-extensions"]' +``` + +Hadoop-client is also needed, you may copy all the hadoop-client dependency jars into directory `druid-thrift-extensions` to make is simple. + + +- Batch Ingestion - `inputFormat` and `tmpjars` should be set. + +This is for batch ingestion using the HadoopDruidIndexer. The inputFormat of inputSpec in ioConfig could be one of `"org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat"` and `com.twitter.elephantbird.mapreduce.input.LzoThriftBlockInputFormat`. Be carefull, when `LzoThriftBlockInputFormat` is used, thrift class must be provided twice. + +```json +{ + "type": "index_hadoop", + "spec": { + "dataSchema": { + "dataSource": "book", + "parser": { + "type": "thrift", + "jarPath": "book.jar", + "thriftClass": "io.druid.data.input.thrift.Book", + "protocol": "compact", + "parseSpec": { + "format": "json", + ... + } + }, + "metricsSpec": [], + "granularitySpec": {} + }, + "ioConfig": { + "type": "hadoop", + "inputSpec": { + "type": "static", + "inputFormat": "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + // "inputFormat": "com.twitter.elephantbird.mapreduce.input.LzoThriftBlockInputFormat", + "paths": "/user/to/some/book.seq" + } + }, + "tuningConfig": { + "type": "hadoop", + "jobProperties": { + "tmpjars":"/user/h_user_profile/du00/druid/test/book.jar", + // "elephantbird.class.for.MultiInputFormat" : "${YOUR_THRIFT_CLASS_NAME}" + } + } + } +} +``` diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index e0bcbd01a69..a9267f10034 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -63,6 +63,7 @@ All of these community extensions can be downloaded using *pull-deps* with the c |sqlserver-metadata-storage|Microsoft SqlServer deep storage.|[link](../development/extensions-contrib/sqlserver.html)| |graphite-emitter|Graphite metrics emitter|[link](../development/extensions-contrib/graphite.html)| |statsd-emitter|StatsD metrics emitter|[link](../development/extensions-contrib/statsd.html)| +|druid-thrift-extensions|Support thrift ingestion |[link](../development/extensions-contrib/thrift.html)| ## Promoting Community Extension to Core Extension diff --git a/extensions-contrib/thrift-extensions/example/books.json b/extensions-contrib/thrift-extensions/example/books.json new file mode 100644 index 00000000000..a866524873d --- /dev/null +++ b/extensions-contrib/thrift-extensions/example/books.json @@ -0,0 +1,65 @@ +{ + "type" : "index_hadoop", + "spec" : { + "dataSchema" : { + "dataSource" : "test", + "parser" : { + "type" : "thrift", + "jarPath" : "example/book.jar", + "thriftClass": "io.druid.data.input.thrift.Book", + "protocol" : "compact", + "parseSpec" : { + "format" : "json", + "timestampSpec" : { + "column" : "date", + "format" : "auto" + }, + "flattenSpec" : { + "useFieldDiscovery" : true, + "fields" : [ { + "type" : "path", + "name" : "lastName", + "expr" : "$.author.lastName" + }, "title" ] + }, + "dimensionsSpec" : { + "dimensions" : [ "title", "lastName" ] + } + } + }, + "metricsSpec" : [ { + "type" : "count", + "name" : "count" + }, { + "type" : "doubleSum", + "name" : "cost", + "fieldName" : "price" + } ], + "granularitySpec" : { + "type" : "uniform", + "segmentGranularity" : "DAY", + "queryGranularity" : "DAY", + "intervals" : [ "2015-09-01/2015-10-01" ] + } + }, + "ioConfig" : { + "type" : "hadoop", + "inputSpec" : { + "type" : "static", + "inputFormat" : "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "paths" : "example/book.seq" + } + }, + "tuningConfig" : { + "type" : "hadoop", + "partitionsSpec" : { + "type" : "hashed", + "targetPartitionSize" : 5000000 + }, + "jobProperties" : { + "tmpjars": "/user/xxx/druid/test/book.jar" + } + } + }, + "dataSource" : "test" +} diff --git a/extensions-contrib/thrift-extensions/pom.xml b/extensions-contrib/thrift-extensions/pom.xml new file mode 100644 index 00000000000..dd544424c1c --- /dev/null +++ b/extensions-contrib/thrift-extensions/pom.xml @@ -0,0 +1,99 @@ + + + + io.druid.extensions.contrib + druid-thrift-extensions + druid-thrift-extensions + druid-thrift-extensions + + + druid + io.druid + 0.9.3-SNAPSHOT + ../../pom.xml + + 4.0.0 + + + 0.9.3 + 4.8 + + + + + org.apache.thrift + libthrift + ${thrift.version} + + + commons-logging + commons-logging + + + httpclient + org.apache.httpcomponents + + + httpcore + org.apache.httpcomponents + + + + + io.druid + druid-indexing-hadoop + ${project.parent.version} + provided + + + org.apache.hadoop + hadoop-client + provided + + + com.twitter.elephantbird + elephant-bird-core + ${elephantbird.version} + + + com.twitter.elephantbird + elephant-bird-hadoop-compat + ${elephantbird.version} + + + com.twitter + scrooge-core_2.11 + 4.10.0 + test + + + junit + junit + test + + + + + + + com.twitter + scrooge-maven-plugin + 4.11.0 + + java + + + + thrift-test-sources + generate-test-sources + + testCompile + + + + + + + diff --git a/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftDeserialization.java b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftDeserialization.java new file mode 100644 index 00000000000..cf64f88f481 --- /dev/null +++ b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftDeserialization.java @@ -0,0 +1,121 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.thrift; + +import com.google.common.base.Preconditions; +import org.apache.commons.codec.binary.Base64; +import org.apache.thrift.TBase; +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TCompactProtocol; +import org.apache.thrift.protocol.TJSONProtocol; +import org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.protocol.TProtocolUtil; +import org.apache.thrift.protocol.TSimpleJSONProtocol; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ThriftDeserialization +{ + private static final Logger log = LoggerFactory.getLogger(ThriftDeserialization.class); + + + private static final ThreadLocal DESERIALIZER_COMPACT = new ThreadLocal() + { + @Override + protected TDeserializer initialValue() + { + return new TDeserializer(new TCompactProtocol.Factory()); + } + }; + + private static final ThreadLocal DESERIALIZER_BINARY = new ThreadLocal() + { + @Override + protected TDeserializer initialValue() + { + return new TDeserializer(new TBinaryProtocol.Factory()); + } + }; + + private static final ThreadLocal DESERIALIZER_JSON = new ThreadLocal() + { + @Override + protected TDeserializer initialValue() + { + return new TDeserializer(new TJSONProtocol.Factory()); + } + }; + + public static final ThreadLocal SERIALIZER_SIMPLE_JSON = new ThreadLocal() + { + @Override + protected TSerializer initialValue() + { + return new TSerializer(new TSimpleJSONProtocol.Factory()); + } + }; + + + private static final byte[] EMPTY_BYTES = new byte[0]; + + public static byte[] decodeB64IfNeeded(final byte[] src) + { + Preconditions.checkNotNull(src, "src bytes cannot be null"); + if (src.length <= 0) { + return EMPTY_BYTES; + } + final byte last = src[src.length - 1]; + return (0 == last || '}' == last) ? src : Base64.decodeBase64(src); + } + + /** + * Deserializes byte-array into thrift object. + *

+ * Supporting binary, compact and json protocols, + * and the byte array could be or not be encoded by Base64. + * + * @param bytes the byte-array to deserialize + * @param thriftObj the output thrift object + * + * @return the output thrift object, or null if error occurs + */ + public static T detectAndDeserialize(final byte[] bytes, final T thriftObj) throws TException + { + Preconditions.checkNotNull(thriftObj); + try { + final byte[] src = decodeB64IfNeeded(bytes); + final TProtocolFactory protocolFactory = TProtocolUtil.guessProtocolFactory(src, null); + Preconditions.checkNotNull(protocolFactory); + if (protocolFactory instanceof TCompactProtocol.Factory) { + DESERIALIZER_COMPACT.get().deserialize(thriftObj, src); + } else if (protocolFactory instanceof TBinaryProtocol.Factory) { + DESERIALIZER_BINARY.get().deserialize(thriftObj, src); + } else { + DESERIALIZER_JSON.get().deserialize(thriftObj, src); + } + } + catch (final IllegalArgumentException e) { + throw new TException(e); + } + return thriftObj; + } +} diff --git a/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftExtensionsModule.java b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftExtensionsModule.java new file mode 100644 index 00000000000..ab443b65868 --- /dev/null +++ b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftExtensionsModule.java @@ -0,0 +1,48 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.thrift; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.inject.Binder; +import io.druid.initialization.DruidModule; + +import java.util.Arrays; +import java.util.List; + +public class ThriftExtensionsModule implements DruidModule +{ + + @Override + public List getJacksonModules() + { + return Arrays.asList( + new SimpleModule("ThriftInputRowParserModule") + .registerSubtypes( + new NamedType(ThriftInputRowParser.class, "thrift") + ) + ); + } + + @Override + public void configure(Binder binder) + { } +} diff --git a/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftInputRowParser.java b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftInputRowParser.java new file mode 100644 index 00000000000..1a2aef38f98 --- /dev/null +++ b/extensions-contrib/thrift-extensions/src/main/java/io/druid/data/input/thrift/ThriftInputRowParser.java @@ -0,0 +1,155 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.thrift; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import com.metamx.common.IAE; +import com.twitter.elephantbird.mapreduce.io.ThriftWritable; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.ParseSpec; +import org.apache.hadoop.io.BytesWritable; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.ByteBuffer; +import java.util.Map; + +import io.druid.java.util.common.parsers.Parser; + +/** + * 1. load thrift class from classpath or provided jar + * 2. deserialize content bytes and serialize to json + * 3. use JsonSpec to do things left + */ +public class ThriftInputRowParser implements InputRowParser +{ + private final ParseSpec parseSpec; + private final String jarPath; + private final String thriftClassName; + + final private Parser parser; + volatile private Class thriftClass = null; + + @JsonCreator + public ThriftInputRowParser( + @JsonProperty("parseSpec") ParseSpec parseSpec, + @JsonProperty("thriftJar") String jarPath, + @JsonProperty("thriftClass") String thriftClassName + ) + { + this.jarPath = jarPath; + this.thriftClassName = thriftClassName; + Preconditions.checkNotNull(thriftClassName, "thrift class name"); + + this.parseSpec = parseSpec; + parser = parseSpec.makeParser(); + } + + public Class getThriftClass() + throws IOException, ClassNotFoundException, IllegalAccessException, InstantiationException + { + final Class thrift; + if (jarPath != null) { + File jar = new File(jarPath); + URLClassLoader child = new URLClassLoader( + new URL[]{jar.toURI().toURL()}, + this.getClass().getClassLoader() + ); + thrift = (Class) Class.forName(thriftClassName, true, child); + } else { + thrift = (Class) Class.forName(thriftClassName); + } + thrift.newInstance(); + return thrift; + } + + + @Override + public InputRow parse(Object input) + { + // There is a Parser check in phase 2 of mapreduce job, thrift jar may not present in peon side. + // Place it this initialization in constructor will get ClassNotFoundException + try { + if (thriftClass == null) { + thriftClass = getThriftClass(); + } + } + catch (IOException e) { + throw new IAE(e, "failed to load jar [%s]", jarPath); + } + catch (ClassNotFoundException e) { + throw new IAE(e, "class [%s] not found in jar", thriftClassName); + } + catch (InstantiationException | IllegalAccessException e) { + throw new IAE(e, "instantiation thrift instance failed"); + } + + final String json; + try { + if (input instanceof ByteBuffer) { // realtime stream + final byte[] bytes = ((ByteBuffer) input).array(); + TBase o = thriftClass.newInstance(); + ThriftDeserialization.detectAndDeserialize(bytes, o); + json = ThriftDeserialization.SERIALIZER_SIMPLE_JSON.get().toString(o); + } else if (input instanceof BytesWritable) { // sequence file + final byte[] bytes = ((BytesWritable) input).getBytes(); + TBase o = thriftClass.newInstance(); + ThriftDeserialization.detectAndDeserialize(bytes, o); + json = ThriftDeserialization.SERIALIZER_SIMPLE_JSON.get().toString(o); + } else if (input instanceof ThriftWritable) { // LzoBlockThrift file + TBase o = (TBase) ((ThriftWritable) input).get(); + json = ThriftDeserialization.SERIALIZER_SIMPLE_JSON.get().toString(o); + } else { + throw new IAE("unsupport input class of [%s]", input.getClass()); + } + } + catch (IllegalAccessException | InstantiationException | TException e) { + throw new IAE("some thing wrong with your thrift?"); + } + + Map record = parser.parse(json); + + return new MapBasedInputRow( + parseSpec.getTimestampSpec().extractTimestamp(record), + parseSpec.getDimensionsSpec().getDimensionNames(), + record + ); + } + + @Override + public ParseSpec getParseSpec() + { + return parseSpec; + } + + @Override + public InputRowParser withParseSpec(ParseSpec parseSpec) + { + return new ThriftInputRowParser(parseSpec, jarPath, thriftClassName); + } +} diff --git a/extensions-contrib/thrift-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule b/extensions-contrib/thrift-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule new file mode 100755 index 00000000000..3b4c298d46b --- /dev/null +++ b/extensions-contrib/thrift-extensions/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -0,0 +1 @@ +io.druid.data.input.thrift.ThriftExtensionsModule diff --git a/extensions-contrib/thrift-extensions/src/test/java/io/druid/data/input/thrift/ThriftInputRowParserTest.java b/extensions-contrib/thrift-extensions/src/test/java/io/druid/data/input/thrift/ThriftInputRowParserTest.java new file mode 100644 index 00000000000..0ac80cbb3f6 --- /dev/null +++ b/extensions-contrib/thrift-extensions/src/test/java/io/druid/data/input/thrift/ThriftInputRowParserTest.java @@ -0,0 +1,124 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.data.input.thrift; + +import com.google.common.collect.Lists; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.DimensionSchema; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.JSONPathFieldSpec; +import io.druid.data.input.impl.JSONPathFieldType; +import io.druid.data.input.impl.JSONPathSpec; +import io.druid.data.input.impl.ParseSpec; +import io.druid.data.input.impl.StringDimensionSchema; +import io.druid.data.input.impl.TimestampSpec; +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.io.BytesWritable; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TCompactProtocol; +import org.apache.thrift.protocol.TJSONProtocol; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class ThriftInputRowParserTest +{ + + private ParseSpec parseSpec; + + @Before + public void setUp() throws Exception + { + parseSpec = new JSONParseSpec(new TimestampSpec("date", "auto", null), + new DimensionsSpec(Lists.newArrayList( + new StringDimensionSchema("title"), + new StringDimensionSchema("lastName") + ), null, null), + new JSONPathSpec( + true, + Lists.newArrayList( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "title", "title"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "lastName", "$.author.lastName") + ) + ), null + ); + } + + @Test + public void testGetThriftClass() throws Exception + { + ThriftInputRowParser parser1 = new ThriftInputRowParser( + parseSpec, + "example/book.jar", + "io.druid.data.input.thrift.Book" + ); + assertEquals(parser1.getThriftClass().getName(), "io.druid.data.input.thrift.Book"); + + ThriftInputRowParser parser2 = new ThriftInputRowParser(parseSpec, null, "io.druid.data.input.thrift.Book"); + assertEquals(parser2.getThriftClass().getName(), "io.druid.data.input.thrift.Book"); + } + + @Test + public void testParse() throws Exception + { + ThriftInputRowParser parser = new ThriftInputRowParser( + parseSpec, + "example/book.jar", + "io.druid.data.input.thrift.Book" + ); + Book book = new Book().setDate("2016-08-29").setPrice(19.9).setTitle("title") + .setAuthor(new Author().setFirstName("first").setLastName("last")); + + TSerializer serializer; + byte[] bytes; + + // 1. compact + serializer = new TSerializer(new TCompactProtocol.Factory()); + bytes = serializer.serialize(book); + serializationAndTest(parser, bytes); + + // 2. binary + base64 + serializer = new TSerializer(new TBinaryProtocol.Factory()); + serializationAndTest(parser, Base64.encodeBase64(serializer.serialize(book))); + + // 3. json + serializer = new TSerializer(new TJSONProtocol.Factory()); + bytes = serializer.serialize(book); + serializationAndTest(parser, bytes); + } + + public void serializationAndTest(ThriftInputRowParser parser, byte[] bytes) throws TException + { + ByteBuffer buffer = ByteBuffer.wrap(bytes); + + InputRow row1 = parser.parse(buffer); + assertTrue(row1.getDimension("title").get(0).equals("title")); + + InputRow row2 = parser.parse(new BytesWritable(bytes)); + assertTrue(row2.getDimension("lastName").get(0).equals("last")); + } +} diff --git a/extensions-contrib/thrift-extensions/src/test/thrift/book.thrift b/extensions-contrib/thrift-extensions/src/test/thrift/book.thrift new file mode 100644 index 00000000000..2d090686ed9 --- /dev/null +++ b/extensions-contrib/thrift-extensions/src/test/thrift/book.thrift @@ -0,0 +1,13 @@ +namespace java io.druid.data.input.thrift + +struct Author { + 1: string firstName; + 2: string lastName; +} + +struct Book { + 1: string date; + 2: double price; + 3: string title; + 4: Author author; +} diff --git a/pom.xml b/pom.xml index 739c9b0e611..0c3e70f9dd8 100644 --- a/pom.xml +++ b/pom.xml @@ -115,6 +115,7 @@ extensions-contrib/time-min-max extensions-contrib/google-extensions extensions-contrib/virtual-columns + extensions-contrib/thrift-extensions