From f2fc34a5e591189dfecf7741aed9bae294aa3e65 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sat, 16 Nov 2019 00:32:17 -0800 Subject: [PATCH 01/11] add parquet support to native batch --- .../apache/druid/data/input/InputFormat.java | 7 +- .../druid/data/input/impl/FileEntity.java | 2 +- .../impl/InputEntityIteratingReader.java | 4 +- extensions-core/parquet-extensions/pom.xml | 253 +++++++++++++++- .../DruidNativeParquetInputFormat.java | 95 ++++++ .../input/parquet/DruidParquetReader.java | 141 +++++++++ .../parquet/ParquetExtensionsModule.java | 52 +++- .../simple/ParquetGroupFlattenerMaker.java | 2 +- .../input/parquet/BaseParquetReaderTest.java | 64 +++++ .../input/parquet/CompatParquetInputTest.java | 2 +- .../parquet/CompatParquetReaderTest.java | 271 ++++++++++++++++++ .../parquet/DecimalParquetReaderTest.java | 118 ++++++++ .../parquet/FlattenSpecParquetInputTest.java | 2 +- .../parquet/FlattenSpecParquetReaderTest.java | 269 +++++++++++++++++ .../parquet/TimestampsParquetReaderTest.java | 107 +++++++ .../input/parquet/WikiParquetReaderTest.java | 58 ++++ 16 files changed, 1423 insertions(+), 24 deletions(-) create mode 100644 extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidNativeParquetInputFormat.java create mode 100644 extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java create mode 100644 extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java create mode 100644 extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java create mode 100644 extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java create mode 100644 extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java create mode 100644 extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java create mode 100644 extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java diff --git a/core/src/main/java/org/apache/druid/data/input/InputFormat.java b/core/src/main/java/org/apache/druid/data/input/InputFormat.java index 682a7bdaa954..6451d0cebef3 100644 --- a/core/src/main/java/org/apache/druid/data/input/InputFormat.java +++ b/core/src/main/java/org/apache/druid/data/input/InputFormat.java @@ -30,6 +30,7 @@ import org.apache.druid.guice.annotations.UnstableApi; import java.io.File; +import java.io.IOException; /** * InputFormat abstracts the file format of input data. @@ -55,5 +56,9 @@ public interface InputFormat @JsonIgnore boolean isSplittable(); - InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory); + InputEntityReader createReader( + InputRowSchema inputRowSchema, + InputEntity source, + File temporaryDirectory + ) throws IOException; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java b/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java index d48345710a3e..568d6152e5b0 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/FileEntity.java @@ -34,7 +34,7 @@ public class FileEntity implements InputEntity { private final File file; - FileEntity(File file) + public FileEntity(File file) { this.file = file; } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java index 385bc5f14591..dc2184aa101f 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/InputEntityIteratingReader.java @@ -64,8 +64,8 @@ public CloseableIterator read() { return createIterator(entity -> { // InputEntityReader is stateful and so a new one should be created per entity. - final InputEntityReader reader = inputFormat.createReader(inputRowSchema, entity, temporaryDirectory); try { + final InputEntityReader reader = inputFormat.createReader(inputRowSchema, entity, temporaryDirectory); return reader.read(); } catch (IOException e) { @@ -79,8 +79,8 @@ public CloseableIterator sample() { return createIterator(entity -> { // InputEntityReader is stateful and so a new one should be created per entity. - final InputEntityReader reader = inputFormat.createReader(inputRowSchema, entity, temporaryDirectory); try { + final InputEntityReader reader = inputFormat.createReader(inputRowSchema, entity, temporaryDirectory); return reader.sample(); } catch (IOException e) { diff --git a/extensions-core/parquet-extensions/pom.xml b/extensions-core/parquet-extensions/pom.xml index 3c4603602131..22d7d771c36b 100644 --- a/extensions-core/parquet-extensions/pom.xml +++ b/extensions-core/parquet-extensions/pom.xml @@ -137,10 +137,249 @@ ${project.parent.version} provided + + + + + + + org.apache.hadoop - hadoop-client - provided + hadoop-mapreduce-client-core + ${hadoop.compile.version} + compile + + + aopalliance + aopalliance + + + org.apache.commons + commons-compress + + + com.google.guava + guava + + + com.google.inject + guice + + + com.google.inject.extensions + guice-servlet + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + javax.inject + javax + + + io.netty + netty + + + slf4j-log4j12 + org.slf4j + + + org.slf4j + slf4j-api + + + protobuf-java + com.google.protobuf + + + + + org.apache.hadoop + hadoop-common + ${hadoop.compile.version} + compile + + + + org.apache.yetus + audience-annotations + + + commons-codec + commons-codec + + + org.apache.commons + commons-compress + + + commons-io + commons-io + + + commons-lang + commons-lang + + + org.apache.commons + commons-math3 + + + commons-net + commons-net + + + org.apache.curator + curator-client + + + org.apache.curator + curator-framework + + + org.apache.curator + curator-recipes + + + com.google.guava + guava + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.sun.jersey + jersey-core + + + com.sun.jersey + jersey-server + + + javax.servlet.jsp + jsp-api + + + com.google.code.findbugs + jsr305 + + + javax.ws.rs + jsr311-api + + + org.apache.zookeeper + zookeeper + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + com.sun.jersey + jersey-json + + + log4j + log4j + + + jetty-sslengine + org.mortbay.jetty + + + jetty-util + org.mortbay.jetty + + + jackson-core-asl + org.codehaus.jackson + + + jets3t + net.java.dev.jets3t + + + jackson-mapper-asl + org.codehaus.jackson + + + jetty + org.mortbay.jetty + + + gson + com.google.code.gson + + + xmlenc + xmlenc + + + httpclient + org.apache.httpcomponents + + + jsch + com.jcraft + + + protobuf-java + com.google.protobuf + + + commons-collections + commons-collections + + + commons-logging + commons-logging + + + commons-cli + commons-cli + + + commons-digester + commons-digester + + + commons-beanutils-core + commons-beanutils + + + apacheds-kerberos-codec + org.apache.directory.server + + + nimbus-jose-jwt + com.nimbusds + + com.google.code.findbugs @@ -162,16 +401,6 @@ joda-time provided - - org.apache.hadoop - hadoop-mapreduce-client-core - provided - - - org.apache.hadoop - hadoop-common - provided - com.google.inject guice diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidNativeParquetInputFormat.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidNativeParquetInputFormat.java new file mode 100644 index 000000000000..bb455918eb9f --- /dev/null +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidNativeParquetInputFormat.java @@ -0,0 +1,95 @@ +/* + * 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.druid.data.input.parquet; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.NestedInputFormat; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.Objects; + +/** + * heh, DruidParquetInputFormat already exists, so I need another name + */ +public class DruidNativeParquetInputFormat extends NestedInputFormat +{ + private final boolean binaryAsString; + + @JsonCreator + public DruidNativeParquetInputFormat( + @JsonProperty("flattenSpec") @Nullable JSONPathSpec flattenSpec, + @JsonProperty("binaryAsString") @Nullable Boolean binaryAsString + ) + { + super(flattenSpec); + this.binaryAsString = binaryAsString == null ? false : binaryAsString; + } + + @JsonProperty + public boolean getBinaryAsString() + { + return binaryAsString; + } + + @Override + public boolean isSplittable() + { + return false; + } + + @Override + public InputEntityReader createReader( + InputRowSchema inputRowSchema, + InputEntity source, + File temporaryDirectory + ) throws IOException + { + return new DruidParquetReader(inputRowSchema, source, temporaryDirectory, getFlattenSpec(), binaryAsString); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + DruidNativeParquetInputFormat that = (DruidNativeParquetInputFormat) o; + return binaryAsString == that.binaryAsString; + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), binaryAsString); + } +} diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java new file mode 100644 index 000000000000..37deaa2ac7dc --- /dev/null +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java @@ -0,0 +1,141 @@ +/* + * 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.druid.data.input.parquet; + +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.IntermediateRowParsingReader; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.parquet.simple.ParquetGroupFlattenerMaker; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.common.parsers.ObjectFlattener; +import org.apache.druid.java.util.common.parsers.ObjectFlatteners; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.apache.parquet.hadoop.example.GroupReadSupport; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; + +public class DruidParquetReader extends IntermediateRowParsingReader +{ + private final InputRowSchema inputRowSchema; + private final ObjectFlattener flattener; + private final byte[] buffer = new byte[InputEntity.DEFAULT_FETCH_BUFFER_SIZE]; + + private final ParquetReader reader; + private final ParquetMetadata metadata; + private final Closer closer; + + public DruidParquetReader( + InputRowSchema inputRowSchema, + InputEntity source, + File temporaryDirectory, + JSONPathSpec flattenSpec, + boolean binaryAsString + ) throws IOException + { + this.inputRowSchema = inputRowSchema; + this.flattener = ObjectFlatteners.create(flattenSpec, new ParquetGroupFlattenerMaker(binaryAsString)); + + closer = Closer.create(); + final InputEntity.CleanableFile file = closer.register(source.fetch(temporaryDirectory, buffer)); + final Path path = new Path(file.file().toURI()); + + final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + reader = closer.register(ParquetReader.builder(new GroupReadSupport(), path).build()); + metadata = ParquetFileReader.readFooter(new Configuration(), path); + } + finally { + Thread.currentThread().setContextClassLoader(currentClassLoader); + } + } + + @Override + protected CloseableIterator intermediateRowIterator() throws IOException + { + return new CloseableIterator() + { + Group value = null; + + @Override + public boolean hasNext() + { + if (value == null) { + try { + value = reader.read(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + return value != null; + } + + @Override + public Group next() + { + if (value == null) { + throw new NoSuchElementException(); + } + Group currentValue = value; + value = null; + return currentValue; + } + + @Override + public void close() throws IOException + { + closer.close(); + } + }; + } + + @Override + protected List parseInputRows(Group intermediateRow) throws IOException, ParseException + { + return Collections.singletonList( + MapInputRowParser.parse( + inputRowSchema.getTimestampSpec(), + inputRowSchema.getDimensionsSpec(), + flattener.flatten(intermediateRow) + ) + ); + } + + @Override + protected String toJson(Group intermediateRow) throws IOException + { + throw new UnsupportedOperationException(); + } +} diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java index f87aacb54ef3..3c425d37e1d7 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java @@ -23,20 +23,34 @@ import com.fasterxml.jackson.databind.jsontype.NamedType; import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; +import com.google.inject.Inject; +import com.google.inject.TypeLiteral; import org.apache.druid.data.input.parquet.avro.ParquetAvroHadoopInputRowParser; import org.apache.druid.data.input.parquet.simple.ParquetHadoopInputRowParser; import org.apache.druid.data.input.parquet.simple.ParquetParseSpec; import org.apache.druid.initialization.DruidModule; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.Properties; public class ParquetExtensionsModule implements DruidModule { - public static final String PARQUET_SIMPLE_INPUT_PARSER_TYPE = "parquet"; - public static final String PARQUET_SIMPLE_PARSE_SPEC_TYPE = "parquet"; - public static final String PARQUET_AVRO_INPUT_PARSER_TYPE = "parquet-avro"; - public static final String PARQUET_AVRO_PARSE_SPEC_TYPE = "avro"; + static final String PARQUET_SIMPLE_INPUT_PARSER_TYPE = "parquet"; + static final String PARQUET_SIMPLE_PARSE_SPEC_TYPE = "parquet"; + static final String PARQUET_AVRO_INPUT_PARSER_TYPE = "parquet-avro"; + static final String PARQUET_AVRO_PARSE_SPEC_TYPE = "avro"; + + private Properties props = null; + + @Inject + public void setProperties(Properties props) + { + this.props = props; + } @Override public List getJacksonModules() @@ -46,7 +60,8 @@ public List getJacksonModules() .registerSubtypes( new NamedType(ParquetAvroHadoopInputRowParser.class, PARQUET_AVRO_INPUT_PARSER_TYPE), new NamedType(ParquetHadoopInputRowParser.class, PARQUET_SIMPLE_INPUT_PARSER_TYPE), - new NamedType(ParquetParseSpec.class, PARQUET_SIMPLE_INPUT_PARSER_TYPE) + new NamedType(ParquetParseSpec.class, PARQUET_SIMPLE_INPUT_PARSER_TYPE), + new NamedType(DruidNativeParquetInputFormat.class, PARQUET_SIMPLE_PARSE_SPEC_TYPE) ) ); } @@ -54,6 +69,33 @@ public List getJacksonModules() @Override public void configure(Binder binder) { + final Configuration conf = new Configuration(); + + // Set explicit CL. Otherwise it'll try to use thread context CL, which may not have all of our dependencies. + conf.setClassLoader(getClass().getClassLoader()); + + // Ensure that FileSystem class level initialization happens with correct CL + // See https://github.com/apache/incubator-druid/issues/1714 + ClassLoader currCtxCl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); + FileSystem.get(conf); + } + catch (IOException ex) { + throw new RuntimeException(ex); + } + finally { + Thread.currentThread().setContextClassLoader(currCtxCl); + } + + if (props != null) { + for (String propName : props.stringPropertyNames()) { + if (propName.startsWith("hadoop.")) { + conf.set(propName.substring("hadoop.".length()), props.getProperty(propName)); + } + } + } + binder.requestInjection(TypeLiteral.get(Configuration.class), conf); } } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java index 2ba939b162a1..eb9a359c8cb0 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java @@ -41,7 +41,7 @@ public class ParquetGroupFlattenerMaker implements ObjectFlatteners.FlattenerMak private final Configuration jsonPathConfiguration; private final ParquetGroupConverter converter; - ParquetGroupFlattenerMaker(boolean binaryAsString) + public ParquetGroupFlattenerMaker(boolean binaryAsString) { this.converter = new ParquetGroupConverter(binaryAsString); this.jsonPathConfiguration = Configuration.builder() diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java new file mode 100644 index 000000000000..48f83c9c37a6 --- /dev/null +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java @@ -0,0 +1,64 @@ +/* + * 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.druid.data.input.parquet; + +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.FileEntity; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class BaseParquetReaderTest +{ + InputEntityReader createReader(String parquetFile, InputRowSchema schema, JSONPathSpec flattenSpec) throws IOException + { + return createReader(parquetFile, schema, flattenSpec, false); + } + + InputEntityReader createReader( + String parquetFile, + InputRowSchema schema, + JSONPathSpec flattenSpec, + boolean binaryAsString + ) throws IOException + { + FileEntity entity = new FileEntity(new File(parquetFile)); + DruidNativeParquetInputFormat parquet = new DruidNativeParquetInputFormat(flattenSpec, binaryAsString); + InputEntityReader reader = parquet.createReader(schema, entity, null); + return reader; + } + + List readAllRows(InputEntityReader reader) throws IOException + { + List rows = new ArrayList<>(); + try (CloseableIterator iterator = reader.read()) { + while (iterator.hasNext()) { + rows.add(iterator.next()); + } + } + return rows; + } +} diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetInputTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetInputTest.java index 2169d9818b8a..6989d09b0d7b 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetInputTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetInputTest.java @@ -67,7 +67,7 @@ public void testBinaryAsString() throws IOException, InterruptedException InputRow row = ((List) config.getParser().parseBatch(data)).get(0); - // without binaryAsString: true, the value would something like "[104, 101, 121, 32, 116, 104, 105, 115, 32, 105, 115, 3.... ]" + // without binaryAsString: true, the value would be "aGV5IHRoaXMgaXMgJsOpKC3DqF/Dp8OgKT1eJMO5KiEgzqleXg==" Assert.assertEquals("hey this is &é(-è_çà)=^$ù*! Ω^^", row.getDimension("field").get(0)); Assert.assertEquals(1471800234, row.getTimestampFromEpoch()); } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java new file mode 100644 index 000000000000..4a14d8e0557b --- /dev/null +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java @@ -0,0 +1,271 @@ +/* + * 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.druid.data.input.parquet; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** + * Duplicate of {@link CompatParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + */ +public class CompatParquetReaderTest extends BaseParquetReaderTest +{ + @Test + public void testBinaryAsString() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("field"))), + ImmutableList.of() + ); + InputEntityReader reader = createReader( + "example/compat/284a0e001476716b-56d5676f53bd6e85_115466471_data.0.parq", + schema, + JSONPathSpec.DEFAULT, + true + ); + + List rows = readAllRows(reader); + + // without binaryAsString: true, the value would be "aGV5IHRoaXMgaXMgJsOpKC3DqF/Dp8OgKT1eJMO5KiEgzqleXg==" + Assert.assertEquals("hey this is &é(-è_çà)=^$ù*! Ω^^", rows.get(0).getDimension("field").get(0)); + Assert.assertEquals(1471800234, rows.get(0).getTimestampFromEpoch()); + } + + + @Test + public void testParquet1217() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + ImmutableList.of("metric1") + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "col", "col"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "metric1", "$.col") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/compat/parquet-1217.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + + Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(0).getTimestamp().toString()); + Assert.assertEquals("-1", rows.get(0).getDimension("col").get(0)); + Assert.assertEquals(-1, rows.get(0).getMetric("metric1")); + Assert.assertTrue(rows.get(4).getDimension("col").isEmpty()); + } + + @Test + public void testParquetThriftCompat() throws IOException + { + /* + message ParquetSchema { + required boolean boolColumn; + required int32 byteColumn; + required int32 shortColumn; + required int32 intColumn; + required int64 longColumn; + required double doubleColumn; + required binary binaryColumn (UTF8); + required binary stringColumn (UTF8); + required binary enumColumn (ENUM); + optional boolean maybeBoolColumn; + optional int32 maybeByteColumn; + optional int32 maybeShortColumn; + optional int32 maybeIntColumn; + optional int64 maybeLongColumn; + optional double maybeDoubleColumn; + optional binary maybeBinaryColumn (UTF8); + optional binary maybeStringColumn (UTF8); + optional binary maybeEnumColumn (ENUM); + required group stringsColumn (LIST) { + repeated binary stringsColumn_tuple (UTF8); + } + required group intSetColumn (LIST) { + repeated int32 intSetColumn_tuple; + } + required group intToStringColumn (MAP) { + repeated group map (MAP_KEY_VALUE) { + required int32 key; + optional binary value (UTF8); + } + } + required group complexColumn (MAP) { + repeated group map (MAP_KEY_VALUE) { + required int32 key; + optional group value (LIST) { + repeated group value_tuple { + required group nestedIntsColumn (LIST) { + repeated int32 nestedIntsColumn_tuple; + } + required binary nestedStringColumn (UTF8); + } + } + } + } + } + */ + + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + Collections.emptyList() + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.PATH, "extractByLogicalMap", "$.intToStringColumn.1"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "extractByComplexLogicalMap", "$.complexColumn.1[0].nestedIntsColumn[1]") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/compat/parquet-thrift-compat.snappy.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + + Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(0).getTimestamp().toString()); + Assert.assertEquals("true", rows.get(0).getDimension("boolColumn").get(0)); + Assert.assertEquals("0", rows.get(0).getDimension("byteColumn").get(0)); + Assert.assertEquals("1", rows.get(0).getDimension("shortColumn").get(0)); + Assert.assertEquals("2", rows.get(0).getDimension("intColumn").get(0)); + Assert.assertEquals("0", rows.get(0).getDimension("longColumn").get(0)); + Assert.assertEquals("0.2", rows.get(0).getDimension("doubleColumn").get(0)); + Assert.assertEquals("val_0", rows.get(0).getDimension("binaryColumn").get(0)); + Assert.assertEquals("val_0", rows.get(0).getDimension("stringColumn").get(0)); + Assert.assertEquals("SPADES", rows.get(0).getDimension("enumColumn").get(0)); + Assert.assertTrue(rows.get(0).getDimension("maybeBoolColumn").isEmpty()); + Assert.assertTrue(rows.get(0).getDimension("maybeByteColumn").isEmpty()); + Assert.assertTrue(rows.get(0).getDimension("maybeShortColumn").isEmpty()); + Assert.assertTrue(rows.get(0).getDimension("maybeIntColumn").isEmpty()); + Assert.assertTrue(rows.get(0).getDimension("maybeLongColumn").isEmpty()); + Assert.assertTrue(rows.get(0).getDimension("maybeDoubleColumn").isEmpty()); + Assert.assertTrue(rows.get(0).getDimension("maybeBinaryColumn").isEmpty()); + Assert.assertTrue(rows.get(0).getDimension("maybeStringColumn").isEmpty()); + Assert.assertTrue(rows.get(0).getDimension("maybeEnumColumn").isEmpty()); + Assert.assertEquals("arr_0", rows.get(0).getDimension("stringsColumn").get(0)); + Assert.assertEquals("arr_1", rows.get(0).getDimension("stringsColumn").get(1)); + Assert.assertEquals("0", rows.get(0).getDimension("intSetColumn").get(0)); + Assert.assertEquals("val_1", rows.get(0).getDimension("extractByLogicalMap").get(0)); + Assert.assertEquals("1", rows.get(0).getDimension("extractByComplexLogicalMap").get(0)); + } + + @Test + public void testOldRepeatedInt() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("repeatedInt"))), + Collections.emptyList() + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "repeatedInt", "repeatedInt") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/compat/old-repeated-int.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(0).getTimestamp().toString()); + Assert.assertEquals("1", rows.get(0).getDimension("repeatedInt").get(0)); + Assert.assertEquals("2", rows.get(0).getDimension("repeatedInt").get(1)); + Assert.assertEquals("3", rows.get(0).getDimension("repeatedInt").get(2)); + } + + + @Test + public void testReadNestedArrayStruct() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("i32_dec", "extracted1", "extracted2"))), + Collections.emptyList() + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.PATH, "extracted1", "$.myComplex[0].id"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "extracted2", "$.myComplex[0].repeatedMessage[*].someId") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/compat/nested-array-struct.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(1).getTimestamp().toString()); + Assert.assertEquals("5", rows.get(1).getDimension("primitive").get(0)); + Assert.assertEquals("4", rows.get(1).getDimension("extracted1").get(0)); + Assert.assertEquals("6", rows.get(1).getDimension("extracted2").get(0)); + } + + @Test + public void testProtoStructWithArray() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + Collections.emptyList() + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.PATH, "extractedOptional", "$.optionalMessage.someId"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "extractedRequired", "$.requiredMessage.someId"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "extractedRepeated", "$.repeatedMessage[*]") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/compat/proto-struct-with-array.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(0).getTimestamp().toString()); + Assert.assertEquals("10", rows.get(0).getDimension("optionalPrimitive").get(0)); + Assert.assertEquals("9", rows.get(0).getDimension("requiredPrimitive").get(0)); + Assert.assertTrue(rows.get(0).getDimension("repeatedPrimitive").isEmpty()); + Assert.assertTrue(rows.get(0).getDimension("extractedOptional").isEmpty()); + Assert.assertEquals("9", rows.get(0).getDimension("extractedRequired").get(0)); + Assert.assertEquals("9", rows.get(0).getDimension("extractedRepeated").get(0)); + Assert.assertEquals("10", rows.get(0).getDimension("extractedRepeated").get(1)); + } +} diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java new file mode 100644 index 000000000000..b10be78fcf89 --- /dev/null +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java @@ -0,0 +1,118 @@ +/* + * 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.druid.data.input.parquet; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; + +/** + * Duplicate of {@link DecimalParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + */ +public class DecimalParquetReaderTest extends BaseParquetReaderTest +{ + @Test + public void testReadParquetDecimalFixedLen() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("fixed_len_dec"))), + ImmutableList.of("metric1") + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "fixed_len_dec", "fixed_len_dec"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "metric1", "$.fixed_len_dec") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/decimals/dec-in-fixed-len.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(1).getTimestamp().toString()); + Assert.assertEquals("1.0", rows.get(1).getDimension("fixed_len_dec").get(0)); + Assert.assertEquals(new BigDecimal("1.0"), rows.get(1).getMetric("metric1")); + } + + @Test + public void testReadParquetDecimali32() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("i32_dec"))), + ImmutableList.of("metric1") + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "i32_dec", "i32_dec"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "metric1", "$.i32_dec") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/decimals/dec-in-i32.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(1).getTimestamp().toString()); + Assert.assertEquals("100", rows.get(1).getDimension("i32_dec").get(0)); + Assert.assertEquals(new BigDecimal(100), rows.get(1).getMetric("metric1")); + } + + @Test + public void testReadParquetDecimali64() throws IOException, InterruptedException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("i64_dec"))), + ImmutableList.of("metric1") + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "i32_dec", "i64_dec"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "metric1", "$.i64_dec") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/decimals/dec-in-i64.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(1).getTimestamp().toString()); + Assert.assertEquals("100", rows.get(1).getDimension("i64_dec").get(0)); + Assert.assertEquals(new BigDecimal(100), rows.get(1).getMetric("metric1")); + } +} diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetInputTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetInputTest.java index 748086dbff78..6532ab222df8 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetInputTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetInputTest.java @@ -36,7 +36,7 @@ @RunWith(Parameterized.class) public class FlattenSpecParquetInputTest extends BaseParquetInputTest { - private static final String TS1 = "2018-09-18T00:18:00.023Z"; + static final String TS1 = "2018-09-18T00:18:00.023Z"; @Parameterized.Parameters(name = "type = {0}") public static Iterable constructorFeeder() diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java new file mode 100644 index 000000000000..b14989c50fd7 --- /dev/null +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java @@ -0,0 +1,269 @@ +/* + * 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.druid.data.input.parquet; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldSpec; +import org.apache.druid.java.util.common.parsers.JSONPathFieldType; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** + * Duplicate of {@link FlattenSpecParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + */ +public class FlattenSpecParquetReaderTest extends BaseParquetReaderTest +{ + @Test + public void testFlat1NoFlattenSpec() throws IOException, InterruptedException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3", "listDim"))), + ImmutableList.of("metric1", "metric2") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(false, ImmutableList.of()); + InputEntityReader reader = createReader( + "example/flattening/test_flat_1.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals("d1v1", rows.get(0).getDimension("dim1").get(0)); + Assert.assertEquals("d2v1", rows.get(0).getDimension("dim2").get(0)); + Assert.assertEquals("1", rows.get(0).getDimension("dim3").get(0)); + Assert.assertEquals("listDim1v1", rows.get(0).getDimension("listDim").get(0)); + Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listDim").get(1)); + Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + } + + @Test + public void testFlat1Autodiscover() throws IOException, InterruptedException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + ImmutableList.of("metric1", "metric2") + ); + InputEntityReader reader = createReader( + "example/flattening/test_flat_1.parquet", + schema, + JSONPathSpec.DEFAULT + ); + + List rows = readAllRows(reader); + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals("d1v1", rows.get(0).getDimension("dim1").get(0)); + Assert.assertEquals("d2v1", rows.get(0).getDimension("dim2").get(0)); + Assert.assertEquals("1", rows.get(0).getDimension("dim3").get(0)); + Assert.assertEquals("listDim1v1", rows.get(0).getDimension("listDim").get(0)); + Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listDim").get(1)); + Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + } + + @Test + public void testFlat1Flatten() throws IOException, InterruptedException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3", "list"))), + ImmutableList.of("metric1", "metric2") + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "timestamp", null), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "dim1", null), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "dim2", null), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "dim3", null), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "list", "$.listDim") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(false, flattenExpr); + InputEntityReader reader = createReader( + "example/flattening/test_flat_1.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals("d1v1", rows.get(0).getDimension("dim1").get(0)); + Assert.assertEquals("d2v1", rows.get(0).getDimension("dim2").get(0)); + Assert.assertEquals("1", rows.get(0).getDimension("dim3").get(0)); + Assert.assertEquals("listDim1v1", rows.get(0).getDimension("list").get(0)); + Assert.assertEquals("listDim1v2", rows.get(0).getDimension("list").get(1)); + Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + } + + @Test + public void testFlat1FlattenSelectListItem() throws IOException, InterruptedException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "listExtracted"))), + ImmutableList.of("metric1", "metric2") + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "timestamp", null), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "dim1", null), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "dim2", null), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "listExtracted", "$.listDim[1]") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(false, flattenExpr); + InputEntityReader reader = createReader( + "example/flattening/test_flat_1.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals("d1v1", rows.get(0).getDimension("dim1").get(0)); + Assert.assertEquals("d2v1", rows.get(0).getDimension("dim2").get(0)); + Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listExtracted").get(0)); + Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + } + + + @Test + public void testNested1NoFlattenSpec() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1"))), + ImmutableList.of("metric1") + ); + InputEntityReader reader = createReader( + "example/flattening/test_nested_1.parquet", + schema, + new JSONPathSpec(false, ImmutableList.of()) + ); + + List rows = readAllRows(reader); + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals("d1v1", rows.get(0).getDimension("dim1").get(0)); + List dims = rows.get(0).getDimensions(); + Assert.assertEquals(1, dims.size()); + Assert.assertFalse(dims.contains("dim2")); + Assert.assertFalse(dims.contains("dim3")); + Assert.assertFalse(dims.contains("listDim")); + Assert.assertFalse(dims.contains("nestedData")); + Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + } + + @Test + public void testNested1Autodiscover() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + ImmutableList.of("metric1", "metric2") + ); + InputEntityReader reader = createReader( + "example/flattening/test_nested_1.parquet", + schema, + JSONPathSpec.DEFAULT + ); + + List rows = readAllRows(reader); + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals("d1v1", rows.get(0).getDimension("dim1").get(0)); + List dims = rows.get(0).getDimensions(); + Assert.assertFalse(dims.contains("dim2")); + Assert.assertFalse(dims.contains("dim3")); + Assert.assertFalse(dims.contains("listDim")); + Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + } + + @Test + public void testNested1Flatten() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + ImmutableList.of("metric1", "metric2") + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "timestamp", null), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "dim1", null), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "dim2", "$.nestedData.dim2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "dim3", "$.nestedData.dim3"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "metric2", "$.nestedData.metric2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "listDim", "$.nestedData.listDim[*]") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/flattening/test_nested_1.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals("d1v1", rows.get(0).getDimension("dim1").get(0)); + Assert.assertEquals("d2v1", rows.get(0).getDimension("dim2").get(0)); + Assert.assertEquals("1", rows.get(0).getDimension("dim3").get(0)); + Assert.assertEquals("listDim1v1", rows.get(0).getDimension("listDim").get(0)); + Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listDim").get(1)); + Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + Assert.assertEquals(2, rows.get(0).getMetric("metric2").longValue()); + } + + @Test + public void testNested1FlattenSelectListItem() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + Collections.emptyList() + ); + List flattenExpr = ImmutableList.of( + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "timestamp", null), + new JSONPathFieldSpec(JSONPathFieldType.ROOT, "dim1", null), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "dim2", "$.nestedData.dim2"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "dim3", "$.nestedData.dim3"), + new JSONPathFieldSpec(JSONPathFieldType.PATH, "listextracted", "$.nestedData.listDim[1]") + ); + JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); + InputEntityReader reader = createReader( + "example/flattening/test_nested_1.parquet", + schema, + flattenSpec + ); + + List rows = readAllRows(reader); + + Assert.assertEquals(FlattenSpecParquetInputTest.TS1, rows.get(0).getTimestamp().toString()); + Assert.assertEquals("d1v1", rows.get(0).getDimension("dim1").get(0)); + Assert.assertEquals("d2v1", rows.get(0).getDimension("dim2").get(0)); + Assert.assertEquals("1", rows.get(0).getDimension("dim3").get(0)); + Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listextracted").get(0)); + Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + } +} diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java new file mode 100644 index 000000000000..c5ac375802fe --- /dev/null +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java @@ -0,0 +1,107 @@ +/* + * 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.druid.data.input.parquet; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** + * Duplicate of {@link TimestampsParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + */ +public class TimestampsParquetReaderTest extends BaseParquetReaderTest +{ + @Test + public void testDateHandling() throws IOException + { + InputRowSchema schemaAsString = new InputRowSchema( + new TimestampSpec("date_as_string", "Y-M-d", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + Collections.emptyList() + ); + InputRowSchema schemaAsDate = new InputRowSchema( + new TimestampSpec("date_as_date", null, null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + Collections.emptyList() + ); + InputEntityReader readerAsString = createReader( + "example/timestamps/test_date_data.snappy.parquet", + schemaAsString, + JSONPathSpec.DEFAULT + ); + InputEntityReader readerAsDate = createReader( + "example/timestamps/test_date_data.snappy.parquet", + schemaAsDate, + JSONPathSpec.DEFAULT + ); + + List rowsWithString = readAllRows(readerAsString); + List rowsWithDate = readAllRows(readerAsDate); + Assert.assertEquals(rowsWithDate.size(), rowsWithString.size()); + + for (int i = 0; i < rowsWithDate.size(); i++) { + Assert.assertEquals(rowsWithString.get(i).getTimestamp(), rowsWithDate.get(i).getTimestamp()); + } + } + + @Test + public void testParseInt96Timestamp() throws IOException + { + // the source parquet file was found in apache spark sql repo tests, where it is known as impala_timestamp.parq + // it has a single column, "ts" which is an int96 timestamp + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("ts", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + Collections.emptyList() + ); + InputEntityReader reader = createReader("example/timestamps/int96_timestamp.parquet", schema, JSONPathSpec.DEFAULT); + + List rows = readAllRows(reader); + Assert.assertEquals("2001-01-01T01:01:01.000Z", rows.get(0).getTimestamp().toString()); + } + + @Test + public void testTimeMillisInInt64() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("time", "auto", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), + Collections.emptyList() + ); + InputEntityReader reader = createReader( + "example/timestamps/timemillis-in-i64.parquet", + schema, + JSONPathSpec.DEFAULT + ); + + List rows = readAllRows(reader); + Assert.assertEquals("1970-01-01T00:00:00.010Z", rows.get(0).getTimestamp().toString()); + } +} diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java new file mode 100644 index 000000000000..72be37cc1774 --- /dev/null +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java @@ -0,0 +1,58 @@ +/* + * 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.druid.data.input.parquet; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** + * Duplicate of {@link WikiParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + */ +public class WikiParquetReaderTest extends BaseParquetReaderTest +{ + @Test + public void testWiki() throws IOException + { + InputRowSchema schema = new InputRowSchema( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("page", "language", "user", "unpatrolled"))), + Collections.emptyList() + ); + InputEntityReader reader = createReader("example/wiki/wiki.parquet", schema, JSONPathSpec.DEFAULT); + + List rows = readAllRows(reader); + Assert.assertEquals("Gypsy Danger", rows.get(0).getDimension("page").get(0)); + String s1 = rows.get(0).getDimension("language").get(0); + String s2 = rows.get(0).getDimension("language").get(1); + Assert.assertEquals("en", s1); + Assert.assertEquals("zh", s2); + } +} From 5c4d1afbe851df1c2875c3a4fc40c6606f0af923 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sat, 16 Nov 2019 00:37:28 -0800 Subject: [PATCH 02/11] cleanup --- extensions-core/parquet-extensions/pom.xml | 7 ------- 1 file changed, 7 deletions(-) diff --git a/extensions-core/parquet-extensions/pom.xml b/extensions-core/parquet-extensions/pom.xml index 22d7d771c36b..392699e3a84c 100644 --- a/extensions-core/parquet-extensions/pom.xml +++ b/extensions-core/parquet-extensions/pom.xml @@ -137,13 +137,6 @@ ${project.parent.version} provided - - - - - - - org.apache.hadoop hadoop-mapreduce-client-core From 968b488c76a3ba4da6e303d17f8486d6eca0116f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 17 Nov 2019 16:52:17 -0800 Subject: [PATCH 03/11] implement toJson for sampler support --- .../input/parquet/DruidParquetReader.java | 43 ++++- .../parquet/simple/ParquetGroupConverter.java | 24 ++- .../simple/ParquetGroupFlattenerMaker.java | 13 +- .../simple/ParquetGroupJsonProvider.java | 2 +- .../input/parquet/BaseParquetReaderTest.java | 12 ++ .../parquet/CompatParquetReaderTest.java | 155 +++++++++++++++++- .../parquet/DecimalParquetReaderTest.java | 43 ++++- .../parquet/FlattenSpecParquetReaderTest.java | 113 ++++++++++++- .../parquet/TimestampsParquetReaderTest.java | 56 ++++++- .../input/parquet/WikiParquetReaderTest.java | 24 +++ 10 files changed, 444 insertions(+), 41 deletions(-) diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java index 37deaa2ac7dc..3a0a2b5c796e 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java @@ -24,7 +24,9 @@ import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.IntermediateRowParsingReader; import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.parquet.simple.ParquetGroupConverter; import org.apache.druid.data.input.parquet.simple.ParquetGroupFlattenerMaker; +import org.apache.druid.data.input.parquet.simple.ParquetGroupJsonProvider; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -41,8 +43,11 @@ import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; public class DruidParquetReader extends IntermediateRowParsingReader @@ -51,6 +56,9 @@ public class DruidParquetReader extends IntermediateRowParsingReader private final ObjectFlattener flattener; private final byte[] buffer = new byte[InputEntity.DEFAULT_FETCH_BUFFER_SIZE]; + private final ParquetGroupConverter converter; + private final ParquetGroupJsonProvider jsonProvider; + private final ParquetReader reader; private final ParquetMetadata metadata; private final Closer closer; @@ -64,6 +72,8 @@ public DruidParquetReader( ) throws IOException { this.inputRowSchema = inputRowSchema; + this.converter = new ParquetGroupConverter(binaryAsString); + this.jsonProvider = new ParquetGroupJsonProvider(converter); this.flattener = ObjectFlatteners.create(flattenSpec, new ParquetGroupFlattenerMaker(binaryAsString)); closer = Closer.create(); @@ -136,6 +146,37 @@ protected List parseInputRows(Group intermediateRow) throws IOExceptio @Override protected String toJson(Group intermediateRow) throws IOException { - throw new UnsupportedOperationException(); + Object converted = convertObject(intermediateRow); + return DEFAULT_JSON_WRITER.writeValueAsString(converted); + } + + private Object convertObject(Object o) + { + if (jsonProvider.isMap(o)) { + Map actualMap = new HashMap<>(); + for (String key : jsonProvider.getPropertyKeys(o)) { + Object field = jsonProvider.getMapValue(o, key); + if (jsonProvider.isMap(field) || jsonProvider.isArray(field)) { + actualMap.put(key, convertObject(converter.finalizeConversion(field))); + } else { + actualMap.put(key, converter.finalizeConversion(field)); + } + } + return actualMap; + } else if (jsonProvider.isArray(o)) { + final int length = jsonProvider.length(o); + List actualList = new ArrayList<>(length); + for (int i = 0; i < length; i++) { + Object element = jsonProvider.getArrayIndex(o, i); + if (jsonProvider.isMap(element) || jsonProvider.isArray(element)) { + actualList.add(convertObject(converter.finalizeConversion(element))); + } else { + actualList.add(converter.finalizeConversion(element)); + } + } + return converter.finalizeConversion(actualList); + } + // unknown, just pass it through + return o; } } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java index 24d5f556475c..78898c09f0a8 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java @@ -42,9 +42,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; -class ParquetGroupConverter +public class ParquetGroupConverter { private static final int JULIAN_EPOCH_OFFSET_DAYS = 2_440_588; private static final long MILLIS_IN_DAY = TimeUnit.DAYS.toMillis(1); @@ -469,7 +471,7 @@ static boolean isWrappedListPrimitive(Object o) private final boolean binaryAsString; - ParquetGroupConverter(boolean binaryAsString) + public ParquetGroupConverter(boolean binaryAsString) { this.binaryAsString = binaryAsString; } @@ -482,7 +484,7 @@ static boolean isWrappedListPrimitive(Object o) * if a field is not present, this method will return null. */ @Nullable - Object convertField(Group g, String fieldName) + public Object convertField(Group g, String fieldName) { return convertField(g, fieldName, binaryAsString); } @@ -492,10 +494,24 @@ Object convertField(Group g, String fieldName) * return lists which contain 'wrapped' primitives, that are a {@link Group} with a single, primitive field (see * {@link ParquetGroupConverter#isWrappedListPrimitive(Object)}) */ - Object unwrapListPrimitive(Object o) + public Object unwrapListPrimitive(Object o) { assert isWrappedListPrimitive(o); Group g = (Group) o; return convertPrimitiveField(g, 0, binaryAsString); } + + public Object finalizeConversion(Object o) + { + // conversion can leave 'wrapped' list primitives + if (isWrappedListPrimitive(o)) { + return unwrapListPrimitive(o); + } else if (o instanceof List) { + List asList = ((List) o).stream().filter(Objects::nonNull).collect(Collectors.toList()); + if (asList.stream().allMatch(ParquetGroupConverter::isWrappedListPrimitive)) { + return asList.stream().map(Group.class::cast).map(this::unwrapListPrimitive).collect(Collectors.toList()); + } + } + return o; + } } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java index eb9a359c8cb0..b091e172aa49 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java @@ -29,8 +29,6 @@ import javax.annotation.Nullable; import java.util.EnumSet; -import java.util.List; -import java.util.Objects; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -97,15 +95,6 @@ public Function makeJsonQueryExtractor(String expr) */ private Object finalizeConversion(Object o) { - // conversion can leave 'wrapped' list primitives - if (ParquetGroupConverter.isWrappedListPrimitive(o)) { - return converter.unwrapListPrimitive(o); - } else if (o instanceof List) { - List asList = ((List) o).stream().filter(Objects::nonNull).collect(Collectors.toList()); - if (asList.stream().allMatch(ParquetGroupConverter::isWrappedListPrimitive)) { - return asList.stream().map(Group.class::cast).map(converter::unwrapListPrimitive).collect(Collectors.toList()); - } - } - return o; + return converter.finalizeConversion(o); } } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupJsonProvider.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupJsonProvider.java index b422e97d7edb..3ba15e41a98e 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupJsonProvider.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupJsonProvider.java @@ -39,7 +39,7 @@ public class ParquetGroupJsonProvider implements JsonProvider { private final ParquetGroupConverter converter; - ParquetGroupJsonProvider(ParquetGroupConverter converter) + public ParquetGroupJsonProvider(ParquetGroupConverter converter) { this.converter = converter; } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java index 48f83c9c37a6..69010a198170 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java @@ -21,6 +21,7 @@ import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.FileEntity; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -61,4 +62,15 @@ List readAllRows(InputEntityReader reader) throws IOException } return rows; } + + List sampleAllRows(InputEntityReader reader) throws IOException + { + List rows = new ArrayList<>(); + try (CloseableIterator iterator = reader.sample()) { + while (iterator.hasNext()) { + rows.add(iterator.next()); + } + } + return rows; + } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java index 4a14d8e0557b..f49e3e4923aa 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -44,13 +45,14 @@ public class CompatParquetReaderTest extends BaseParquetReaderTest @Test public void testBinaryAsString() throws IOException { + final String file = "example/compat/284a0e001476716b-56d5676f53bd6e85_115466471_data.0.parq"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("ts", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("field"))), ImmutableList.of() ); InputEntityReader reader = createReader( - "example/compat/284a0e001476716b-56d5676f53bd6e85_115466471_data.0.parq", + file, schema, JSONPathSpec.DEFAULT, true @@ -61,12 +63,26 @@ public void testBinaryAsString() throws IOException // without binaryAsString: true, the value would be "aGV5IHRoaXMgaXMgJsOpKC3DqF/Dp8OgKT1eJMO5KiEgzqleXg==" Assert.assertEquals("hey this is &é(-è_çà)=^$ù*! Ω^^", rows.get(0).getDimension("field").get(0)); Assert.assertEquals(1471800234, rows.get(0).getTimestampFromEpoch()); + + reader = createReader( + file, + schema, + JSONPathSpec.DEFAULT, + true + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"field\" : \"hey this is &é(-è_çà)=^$ù*! Ω^^\",\n" + + " \"ts\" : 1471800234\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); } @Test public void testParquet1217() throws IOException { + final String file = "example/compat/parquet-1217.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), @@ -78,7 +94,7 @@ public void testParquet1217() throws IOException ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/compat/parquet-1217.parquet", + file, schema, flattenSpec ); @@ -89,6 +105,17 @@ public void testParquet1217() throws IOException Assert.assertEquals("-1", rows.get(0).getDimension("col").get(0)); Assert.assertEquals(-1, rows.get(0).getMetric("metric1")); Assert.assertTrue(rows.get(4).getDimension("col").isEmpty()); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"col\" : -1\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); } @Test @@ -141,7 +168,7 @@ required group nestedIntsColumn (LIST) { } } */ - + final String file = "example/compat/parquet-thrift-compat.snappy.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), @@ -153,7 +180,7 @@ required group nestedIntsColumn (LIST) { ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/compat/parquet-thrift-compat.snappy.parquet", + file, schema, flattenSpec ); @@ -184,11 +211,79 @@ required group nestedIntsColumn (LIST) { Assert.assertEquals("0", rows.get(0).getDimension("intSetColumn").get(0)); Assert.assertEquals("val_1", rows.get(0).getDimension("extractByLogicalMap").get(0)); Assert.assertEquals("1", rows.get(0).getDimension("extractByComplexLogicalMap").get(0)); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"enumColumn\" : \"SPADES\",\n" + + " \"maybeStringColumn\" : { },\n" + + " \"maybeBinaryColumn\" : { },\n" + + " \"shortColumn\" : 1,\n" + + " \"byteColumn\" : 0,\n" + + " \"maybeBoolColumn\" : { },\n" + + " \"intColumn\" : 2,\n" + + " \"doubleColumn\" : 0.2,\n" + + " \"maybeByteColumn\" : { },\n" + + " \"intSetColumn\" : [ 0 ],\n" + + " \"boolColumn\" : true,\n" + + " \"binaryColumn\" : \"val_0\",\n" + + " \"maybeIntColumn\" : { },\n" + + " \"intToStringColumn\" : {\n" + + " \"0\" : \"val_0\",\n" + + " \"1\" : \"val_1\",\n" + + " \"2\" : \"val_2\"\n" + + " },\n" + + " \"maybeDoubleColumn\" : { },\n" + + " \"maybeEnumColumn\" : { },\n" + + " \"maybeLongColumn\" : { },\n" + + " \"stringsColumn\" : [ \"arr_0\", \"arr_1\", \"arr_2\" ],\n" + + " \"longColumn\" : 0,\n" + + " \"stringColumn\" : \"val_0\",\n" + + " \"maybeShortColumn\" : { },\n" + + " \"complexColumn\" : {\n" + + " \"0\" : [ {\n" + + " \"nestedStringColumn\" : \"val_0\",\n" + + " \"nestedIntsColumn\" : [ 0, 1, 2 ]\n" + + " }, {\n" + + " \"nestedStringColumn\" : \"val_1\",\n" + + " \"nestedIntsColumn\" : [ 1, 2, 3 ]\n" + + " }, {\n" + + " \"nestedStringColumn\" : \"val_2\",\n" + + " \"nestedIntsColumn\" : [ 2, 3, 4 ]\n" + + " } ],\n" + + " \"1\" : [ {\n" + + " \"nestedStringColumn\" : \"val_0\",\n" + + " \"nestedIntsColumn\" : [ 0, 1, 2 ]\n" + + " }, {\n" + + " \"nestedStringColumn\" : \"val_1\",\n" + + " \"nestedIntsColumn\" : [ 1, 2, 3 ]\n" + + " }, {\n" + + " \"nestedStringColumn\" : \"val_2\",\n" + + " \"nestedIntsColumn\" : [ 2, 3, 4 ]\n" + + " } ],\n" + + " \"2\" : [ {\n" + + " \"nestedStringColumn\" : \"val_0\",\n" + + " \"nestedIntsColumn\" : [ 0, 1, 2 ]\n" + + " }, {\n" + + " \"nestedStringColumn\" : \"val_1\",\n" + + " \"nestedIntsColumn\" : [ 1, 2, 3 ]\n" + + " }, {\n" + + " \"nestedStringColumn\" : \"val_2\",\n" + + " \"nestedIntsColumn\" : [ 2, 3, 4 ]\n" + + " } ]\n" + + " }\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); } @Test public void testOldRepeatedInt() throws IOException { + final String file = "example/compat/old-repeated-int.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("repeatedInt"))), @@ -199,7 +294,7 @@ public void testOldRepeatedInt() throws IOException ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/compat/old-repeated-int.parquet", + file, schema, flattenSpec ); @@ -209,12 +304,24 @@ public void testOldRepeatedInt() throws IOException Assert.assertEquals("1", rows.get(0).getDimension("repeatedInt").get(0)); Assert.assertEquals("2", rows.get(0).getDimension("repeatedInt").get(1)); Assert.assertEquals("3", rows.get(0).getDimension("repeatedInt").get(2)); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"repeatedInt\" : [ 1, 2, 3 ]\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); } @Test public void testReadNestedArrayStruct() throws IOException { + final String file = "example/compat/nested-array-struct.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("i32_dec", "extracted1", "extracted2"))), @@ -226,7 +333,7 @@ public void testReadNestedArrayStruct() throws IOException ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/compat/nested-array-struct.parquet", + file, schema, flattenSpec ); @@ -236,11 +343,27 @@ public void testReadNestedArrayStruct() throws IOException Assert.assertEquals("5", rows.get(1).getDimension("primitive").get(0)); Assert.assertEquals("4", rows.get(1).getDimension("extracted1").get(0)); Assert.assertEquals("6", rows.get(1).getDimension("extracted2").get(0)); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"primitive\" : 2,\n" + + " \"myComplex\" : [ {\n" + + " \"id\" : 1,\n" + + " \"repeatedMessage\" : [ 3 ]\n" + + " } ]\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); } @Test public void testProtoStructWithArray() throws IOException { + final String file = "example/compat/proto-struct-with-array.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), @@ -253,7 +376,7 @@ public void testProtoStructWithArray() throws IOException ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/compat/proto-struct-with-array.parquet", + file, schema, flattenSpec ); @@ -267,5 +390,23 @@ public void testProtoStructWithArray() throws IOException Assert.assertEquals("9", rows.get(0).getDimension("extractedRequired").get(0)); Assert.assertEquals("9", rows.get(0).getDimension("extractedRepeated").get(0)); Assert.assertEquals("10", rows.get(0).getDimension("extractedRepeated").get(1)); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"optionalMessage\" : { },\n" + + " \"requiredPrimitive\" : 9,\n" + + " \"repeatedPrimitive\" : { },\n" + + " \"repeatedMessage\" : [ 9, 10 ],\n" + + " \"optionalPrimitive\" : 10,\n" + + " \"requiredMessage\" : {\n" + + " \"someId\" : 9\n" + + " }\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java index b10be78fcf89..5511eb7f677d 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -44,6 +45,7 @@ public class DecimalParquetReaderTest extends BaseParquetReaderTest @Test public void testReadParquetDecimalFixedLen() throws IOException { + final String file = "example/decimals/dec-in-fixed-len.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("fixed_len_dec"))), @@ -55,7 +57,7 @@ public void testReadParquetDecimalFixedLen() throws IOException ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/decimals/dec-in-fixed-len.parquet", + file, schema, flattenSpec ); @@ -64,11 +66,23 @@ public void testReadParquetDecimalFixedLen() throws IOException Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(1).getTimestamp().toString()); Assert.assertEquals("1.0", rows.get(1).getDimension("fixed_len_dec").get(0)); Assert.assertEquals(new BigDecimal("1.0"), rows.get(1).getMetric("metric1")); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"fixed_len_dec\" : 1.0\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(1).getRawJson()); } @Test public void testReadParquetDecimali32() throws IOException { + final String file = "example/decimals/dec-in-i32.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("i32_dec"))), @@ -80,7 +94,7 @@ public void testReadParquetDecimali32() throws IOException ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/decimals/dec-in-i32.parquet", + file, schema, flattenSpec ); @@ -89,11 +103,23 @@ public void testReadParquetDecimali32() throws IOException Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(1).getTimestamp().toString()); Assert.assertEquals("100", rows.get(1).getDimension("i32_dec").get(0)); Assert.assertEquals(new BigDecimal(100), rows.get(1).getMetric("metric1")); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"i32_dec\" : 100\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(1).getRawJson()); } @Test public void testReadParquetDecimali64() throws IOException, InterruptedException { + final String file = "example/decimals/dec-in-i64.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", DateTimes.of("2018-09-01T00:00:00.000Z")), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("i64_dec"))), @@ -105,7 +131,7 @@ public void testReadParquetDecimali64() throws IOException, InterruptedException ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/decimals/dec-in-i64.parquet", + file, schema, flattenSpec ); @@ -114,5 +140,16 @@ public void testReadParquetDecimali64() throws IOException, InterruptedException Assert.assertEquals("2018-09-01T00:00:00.000Z", rows.get(1).getTimestamp().toString()); Assert.assertEquals("100", rows.get(1).getDimension("i64_dec").get(0)); Assert.assertEquals(new BigDecimal(100), rows.get(1).getMetric("metric1")); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"i64_dec\" : 100\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(1).getRawJson()); } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java index b14989c50fd7..f4bc5d05578a 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -40,9 +41,30 @@ */ public class FlattenSpecParquetReaderTest extends BaseParquetReaderTest { + private static final String FLAT_JSON = "{\n" + + " \"listDim\" : [ \"listDim1v1\", \"listDim1v2\" ],\n" + + " \"dim3\" : 1,\n" + + " \"dim2\" : \"d2v1\",\n" + + " \"dim1\" : \"d1v1\",\n" + + " \"metric1\" : 1,\n" + + " \"timestamp\" : 1537229880023\n" + + "}"; + + private static final String NESTED_JSON = "{\n" + + " \"nestedData\" : {\n" + + " \"listDim\" : [ \"listDim1v1\", \"listDim1v2\" ],\n" + + " \"dim3\" : 1,\n" + + " \"dim2\" : \"d2v1\",\n" + + " \"metric2\" : 2\n" + + " },\n" + + " \"dim1\" : \"d1v1\",\n" + + " \"metric1\" : 1,\n" + + " \"timestamp\" : 1537229880023\n" + + "}"; @Test public void testFlat1NoFlattenSpec() throws IOException, InterruptedException { + final String file = "example/flattening/test_flat_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3", "listDim"))), @@ -50,7 +72,7 @@ public void testFlat1NoFlattenSpec() throws IOException, InterruptedException ); JSONPathSpec flattenSpec = new JSONPathSpec(false, ImmutableList.of()); InputEntityReader reader = createReader( - "example/flattening/test_flat_1.parquet", + file, schema, flattenSpec ); @@ -63,18 +85,27 @@ public void testFlat1NoFlattenSpec() throws IOException, InterruptedException Assert.assertEquals("listDim1v1", rows.get(0).getDimension("listDim").get(0)); Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listDim").get(1)); Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + Assert.assertEquals(FLAT_JSON, sampled.get(0).getRawJson()); } @Test public void testFlat1Autodiscover() throws IOException, InterruptedException { + final String file = "example/flattening/test_flat_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), ImmutableList.of("metric1", "metric2") ); InputEntityReader reader = createReader( - "example/flattening/test_flat_1.parquet", + file, schema, JSONPathSpec.DEFAULT ); @@ -87,11 +118,20 @@ public void testFlat1Autodiscover() throws IOException, InterruptedException Assert.assertEquals("listDim1v1", rows.get(0).getDimension("listDim").get(0)); Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listDim").get(1)); Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + + reader = createReader( + file, + schema, + JSONPathSpec.DEFAULT + ); + List sampled = sampleAllRows(reader); + Assert.assertEquals(FLAT_JSON, sampled.get(0).getRawJson()); } @Test public void testFlat1Flatten() throws IOException, InterruptedException { + final String file = "example/flattening/test_flat_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3", "list"))), @@ -106,7 +146,7 @@ public void testFlat1Flatten() throws IOException, InterruptedException ); JSONPathSpec flattenSpec = new JSONPathSpec(false, flattenExpr); InputEntityReader reader = createReader( - "example/flattening/test_flat_1.parquet", + file, schema, flattenSpec ); @@ -119,11 +159,20 @@ public void testFlat1Flatten() throws IOException, InterruptedException Assert.assertEquals("listDim1v1", rows.get(0).getDimension("list").get(0)); Assert.assertEquals("listDim1v2", rows.get(0).getDimension("list").get(1)); Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + Assert.assertEquals(FLAT_JSON, sampled.get(0).getRawJson()); } @Test public void testFlat1FlattenSelectListItem() throws IOException, InterruptedException { + final String file = "example/flattening/test_flat_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "listExtracted"))), @@ -137,7 +186,7 @@ public void testFlat1FlattenSelectListItem() throws IOException, InterruptedExce ); JSONPathSpec flattenSpec = new JSONPathSpec(false, flattenExpr); InputEntityReader reader = createReader( - "example/flattening/test_flat_1.parquet", + file, schema, flattenSpec ); @@ -148,21 +197,32 @@ public void testFlat1FlattenSelectListItem() throws IOException, InterruptedExce Assert.assertEquals("d2v1", rows.get(0).getDimension("dim2").get(0)); Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listExtracted").get(0)); Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + + Assert.assertEquals(FLAT_JSON, sampled.get(0).getRawJson()); } @Test public void testNested1NoFlattenSpec() throws IOException { + final String file = "example/flattening/test_nested_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1"))), ImmutableList.of("metric1") ); + JSONPathSpec flattenSpec = new JSONPathSpec(false, ImmutableList.of()); InputEntityReader reader = createReader( - "example/flattening/test_nested_1.parquet", + file, schema, - new JSONPathSpec(false, ImmutableList.of()) + flattenSpec ); List rows = readAllRows(reader); @@ -175,18 +235,27 @@ public void testNested1NoFlattenSpec() throws IOException Assert.assertFalse(dims.contains("listDim")); Assert.assertFalse(dims.contains("nestedData")); Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + Assert.assertEquals(NESTED_JSON, sampled.get(0).getRawJson()); } @Test public void testNested1Autodiscover() throws IOException { + final String file = "example/flattening/test_nested_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), ImmutableList.of("metric1", "metric2") ); InputEntityReader reader = createReader( - "example/flattening/test_nested_1.parquet", + file, schema, JSONPathSpec.DEFAULT ); @@ -199,11 +268,20 @@ public void testNested1Autodiscover() throws IOException Assert.assertFalse(dims.contains("dim3")); Assert.assertFalse(dims.contains("listDim")); Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + + reader = createReader( + file, + schema, + JSONPathSpec.DEFAULT + ); + List sampled = sampleAllRows(reader); + Assert.assertEquals(NESTED_JSON, sampled.get(0).getRawJson()); } @Test public void testNested1Flatten() throws IOException { + final String file = "example/flattening/test_nested_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), @@ -219,7 +297,7 @@ public void testNested1Flatten() throws IOException ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/flattening/test_nested_1.parquet", + file, schema, flattenSpec ); @@ -233,11 +311,20 @@ public void testNested1Flatten() throws IOException Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listDim").get(1)); Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); Assert.assertEquals(2, rows.get(0).getMetric("metric2").longValue()); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + Assert.assertEquals(NESTED_JSON, sampled.get(0).getRawJson()); } @Test public void testNested1FlattenSelectListItem() throws IOException { + final String file = "example/flattening/test_nested_1.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), @@ -252,7 +339,7 @@ public void testNested1FlattenSelectListItem() throws IOException ); JSONPathSpec flattenSpec = new JSONPathSpec(true, flattenExpr); InputEntityReader reader = createReader( - "example/flattening/test_nested_1.parquet", + file, schema, flattenSpec ); @@ -265,5 +352,13 @@ public void testNested1FlattenSelectListItem() throws IOException Assert.assertEquals("1", rows.get(0).getDimension("dim3").get(0)); Assert.assertEquals("listDim1v2", rows.get(0).getDimension("listextracted").get(0)); Assert.assertEquals(1, rows.get(0).getMetric("metric1").longValue()); + + reader = createReader( + file, + schema, + flattenSpec + ); + List sampled = sampleAllRows(reader); + Assert.assertEquals(NESTED_JSON, sampled.get(0).getRawJson()); } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java index c5ac375802fe..6ee4173fc140 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -41,6 +42,7 @@ public class TimestampsParquetReaderTest extends BaseParquetReaderTest @Test public void testDateHandling() throws IOException { + final String file = "example/timestamps/test_date_data.snappy.parquet"; InputRowSchema schemaAsString = new InputRowSchema( new TimestampSpec("date_as_string", "Y-M-d", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), @@ -52,12 +54,12 @@ public void testDateHandling() throws IOException Collections.emptyList() ); InputEntityReader readerAsString = createReader( - "example/timestamps/test_date_data.snappy.parquet", + file, schemaAsString, JSONPathSpec.DEFAULT ); InputEntityReader readerAsDate = createReader( - "example/timestamps/test_date_data.snappy.parquet", + file, schemaAsDate, JSONPathSpec.DEFAULT ); @@ -69,6 +71,28 @@ public void testDateHandling() throws IOException for (int i = 0; i < rowsWithDate.size(); i++) { Assert.assertEquals(rowsWithString.get(i).getTimestamp(), rowsWithDate.get(i).getTimestamp()); } + + readerAsString = createReader( + file, + schemaAsString, + JSONPathSpec.DEFAULT + ); + readerAsDate = createReader( + file, + schemaAsDate, + JSONPathSpec.DEFAULT + ); + List sampledAsString = sampleAllRows(readerAsString); + List sampledAsDate = sampleAllRows(readerAsDate); + final String expectedJson = "{\n" + + " \"date_as_string\" : \"2017-06-18\",\n" + + " \"timestamp_as_timestamp\" : 1497702471815,\n" + + " \"timestamp_as_string\" : \"2017-06-17 14:27:51.815\",\n" + + " \"idx\" : 1,\n" + + " \"date_as_date\" : 1497744000000\n" + + "}"; + Assert.assertEquals(expectedJson, sampledAsString.get(0).getRawJson()); + Assert.assertEquals(expectedJson, sampledAsDate.get(0).getRawJson()); } @Test @@ -76,32 +100,56 @@ public void testParseInt96Timestamp() throws IOException { // the source parquet file was found in apache spark sql repo tests, where it is known as impala_timestamp.parq // it has a single column, "ts" which is an int96 timestamp + final String file = "example/timestamps/int96_timestamp.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("ts", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), Collections.emptyList() ); - InputEntityReader reader = createReader("example/timestamps/int96_timestamp.parquet", schema, JSONPathSpec.DEFAULT); + InputEntityReader reader = createReader(file, schema, JSONPathSpec.DEFAULT); List rows = readAllRows(reader); Assert.assertEquals("2001-01-01T01:01:01.000Z", rows.get(0).getTimestamp().toString()); + + reader = createReader( + file, + schema, + JSONPathSpec.DEFAULT + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"ts\" : 978310861000\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); } @Test public void testTimeMillisInInt64() throws IOException { + final String file = "example/timestamps/timemillis-in-i64.parquet"; InputRowSchema schema = new InputRowSchema( new TimestampSpec("time", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of())), Collections.emptyList() ); InputEntityReader reader = createReader( - "example/timestamps/timemillis-in-i64.parquet", + file, schema, JSONPathSpec.DEFAULT ); List rows = readAllRows(reader); Assert.assertEquals("1970-01-01T00:00:00.010Z", rows.get(0).getTimestamp().toString()); + + reader = createReader( + file, + schema, + JSONPathSpec.DEFAULT + ); + List sampled = sampleAllRows(reader); + final String expectedJson = "{\n" + + " \"time\" : 10\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java index 72be37cc1774..b2c1977146ef 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusJson; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -54,5 +55,28 @@ public void testWiki() throws IOException String s2 = rows.get(0).getDimension("language").get(1); Assert.assertEquals("en", s1); Assert.assertEquals("zh", s2); + + reader = createReader("example/wiki/wiki.parquet", schema, JSONPathSpec.DEFAULT); + List sampled = sampleAllRows(reader); + + final String expectedJson = "{\n" + + " \"continent\" : \"North America\",\n" + + " \"country\" : \"United States\",\n" + + " \"added\" : 57,\n" + + " \"city\" : \"San Francisco\",\n" + + " \"unpatrolled\" : \"true\",\n" + + " \"delta\" : -143,\n" + + " \"language\" : [ \"en\", \"zh\" ],\n" + + " \"robot\" : \"false\",\n" + + " \"deleted\" : 200,\n" + + " \"newPage\" : \"true\",\n" + + " \"namespace\" : \"article\",\n" + + " \"anonymous\" : \"false\",\n" + + " \"page\" : \"Gypsy Danger\",\n" + + " \"region\" : \"Bay Area\",\n" + + " \"user\" : \"nuclear\",\n" + + " \"timestamp\" : \"2013-08-31T01:02:33Z\"\n" + + "}"; + Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); } } From 35b46172ef6db41b0ff666c699ae823b5e9a201c Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 17 Nov 2019 16:58:01 -0800 Subject: [PATCH 04/11] better binaryAsString test --- .../parquet/CompatParquetReaderTest.java | 27 ++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java index f49e3e4923aa..f12f3373770f 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java @@ -58,11 +58,23 @@ public void testBinaryAsString() throws IOException true ); + InputEntityReader readerNotAsString = createReader( + file, + schema, + JSONPathSpec.DEFAULT, + false + ); + List rows = readAllRows(reader); + List rowsAsBinary = readAllRows(readerNotAsString); - // without binaryAsString: true, the value would be "aGV5IHRoaXMgaXMgJsOpKC3DqF/Dp8OgKT1eJMO5KiEgzqleXg==" Assert.assertEquals("hey this is &é(-è_çà)=^$ù*! Ω^^", rows.get(0).getDimension("field").get(0)); Assert.assertEquals(1471800234, rows.get(0).getTimestampFromEpoch()); + Assert.assertEquals( + "aGV5IHRoaXMgaXMgJsOpKC3DqF/Dp8OgKT1eJMO5KiEgzqleXg==", + rowsAsBinary.get(0).getDimension("field").get(0) + ); + Assert.assertEquals(1471800234, rowsAsBinary.get(0).getTimestampFromEpoch()); reader = createReader( file, @@ -70,12 +82,25 @@ public void testBinaryAsString() throws IOException JSONPathSpec.DEFAULT, true ); + readerNotAsString = createReader( + file, + schema, + JSONPathSpec.DEFAULT, + false + ); List sampled = sampleAllRows(reader); + List sampledAsBinary = sampleAllRows(readerNotAsString); final String expectedJson = "{\n" + " \"field\" : \"hey this is &é(-è_çà)=^$ù*! Ω^^\",\n" + " \"ts\" : 1471800234\n" + "}"; Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + + final String expectedJsonBinary = "{\n" + + " \"field\" : \"aGV5IHRoaXMgaXMgJsOpKC3DqF/Dp8OgKT1eJMO5KiEgzqleXg==\",\n" + + " \"ts\" : 1471800234\n" + + "}"; + Assert.assertEquals(expectedJsonBinary, sampledAsBinary.get(0).getRawJson()); } From f499519f363784a4b9e511407d545824054b737c Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 17 Nov 2019 21:44:43 -0800 Subject: [PATCH 05/11] docs --- docs/development/extensions-core/parquet.md | 45 +++++++++++++++++-- .../parquet/FlattenSpecParquetReaderTest.java | 9 ++-- 2 files changed, 47 insertions(+), 7 deletions(-) diff --git a/docs/development/extensions-core/parquet.md b/docs/development/extensions-core/parquet.md index 645cf2418d00..b2629b5db391 100644 --- a/docs/development/extensions-core/parquet.md +++ b/docs/development/extensions-core/parquet.md @@ -26,12 +26,51 @@ title: "Apache Parquet Extension" This Apache Druid (incubating) module extends [Druid Hadoop based indexing](../../ingestion/hadoop.md) to ingest data directly from offline Apache Parquet files. -Note: `druid-parquet-extensions` depends on the `druid-avro-extensions` module, so be sure to +Note: If using the `parquet-avro` parser for Apache Hadoop based indexing, `druid-parquet-extensions` depends on the `druid-avro-extensions` module, so be sure to [include both](../../development/extensions.md#loading-extensions). +## Parquet and Native Batch +This extension provides a `parquet` input format which can be used with Druid [native batch ingestion](../../ingestion/native-batch.md). + +### Parquet InputFormat +|Field | Type | Description | Required| +|---|---|---|---| +|type| String| This should be set to `parquet` to read Parquet file| yes | +|flattenSpec| JSON Object |Define a [`flattenSpec`](../../ingestion/index.md#flattenspec) to extract nested values from a Parquet file. Note that only 'path' expression are supported ('jq' is unavailable).| no (default will auto-discover 'root' level properties) | +| binaryAsString | Boolean | Specifies if the bytes parquet column which is not logically marked as a string or enum type should be treated as a UTF-8 encoded string. | no (default == false) | + +### Example + +```json + ... + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "local", + "baseDir": "/some/path/to/file/", + "filter": "file.parquet" + }, + "inputFormat": { + "type": "parquet" + "flattenSpec": { + "useFieldDiscovery": true, + "fields": [ + { + "type": "path", + "name": "nested", + "expr": "$.path.to.nested" + } + ] + } + "binaryAsString": false + }, + ... + } + ... +``` ## Parquet Hadoop Parser -This extension provides two ways to parse Parquet files: +For Hadoop, this extension provides two parser implementations for reading Parquet files: * `parquet` - using a simple conversion contained within this extension * `parquet-avro` - conversion to avro records with the `parquet-avro` library and using the `druid-avro-extensions` @@ -62,7 +101,7 @@ However, `parquet-avro` was the original basis for this extension, and as such i |----------|-------------|----------------------------------------------------------------------------------------|---------| | type | String | Choose `parquet` or `parquet-avro` to determine how Parquet files are parsed | yes | | parseSpec | JSON Object | Specifies the timestamp and dimensions of the data, and optionally, a flatten spec. Valid parseSpec formats are `timeAndDims`, `parquet`, `avro` (if used with avro conversion). | yes | -| binaryAsString | Boolean | Specifies if the bytes parquet column which is not logically marked as a string or enum type should be converted to strings anyway. | no(default == false) | +| binaryAsString | Boolean | Specifies if the bytes parquet column which is not logically marked as a string or enum type should be treated as a UTF-8 encoded string. | no(default == false) | When the time dimension is a [DateType column](https://github.com/apache/parquet-format/blob/master/LogicalTypes.md), a format should not be supplied. When the format is UTF8 (String), either `auto` or a explicitly defined [format](http://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html) is required. diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java index f4bc5d05578a..4a7432aa2cf6 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java @@ -61,8 +61,9 @@ public class FlattenSpecParquetReaderTest extends BaseParquetReaderTest + " \"metric1\" : 1,\n" + " \"timestamp\" : 1537229880023\n" + "}"; + @Test - public void testFlat1NoFlattenSpec() throws IOException, InterruptedException + public void testFlat1NoFlattenSpec() throws IOException { final String file = "example/flattening/test_flat_1.parquet"; InputRowSchema schema = new InputRowSchema( @@ -96,7 +97,7 @@ public void testFlat1NoFlattenSpec() throws IOException, InterruptedException } @Test - public void testFlat1Autodiscover() throws IOException, InterruptedException + public void testFlat1Autodiscover() throws IOException { final String file = "example/flattening/test_flat_1.parquet"; InputRowSchema schema = new InputRowSchema( @@ -129,7 +130,7 @@ public void testFlat1Autodiscover() throws IOException, InterruptedException } @Test - public void testFlat1Flatten() throws IOException, InterruptedException + public void testFlat1Flatten() throws IOException { final String file = "example/flattening/test_flat_1.parquet"; InputRowSchema schema = new InputRowSchema( @@ -170,7 +171,7 @@ public void testFlat1Flatten() throws IOException, InterruptedException } @Test - public void testFlat1FlattenSelectListItem() throws IOException, InterruptedException + public void testFlat1FlattenSelectListItem() throws IOException { final String file = "example/flattening/test_flat_1.parquet"; InputRowSchema schema = new InputRowSchema( From d80ca292a578ed3ba5e7cde6bc3bd42bb6129826 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 18 Nov 2019 00:17:34 -0800 Subject: [PATCH 06/11] i hate spellcheck --- website/.spelling | 1 + 1 file changed, 1 insertion(+) diff --git a/website/.spelling b/website/.spelling index c177e709b17b..83bc86b38200 100644 --- a/website/.spelling +++ b/website/.spelling @@ -90,6 +90,7 @@ ISO8601 IndexSpec IndexTask InfluxDB +InputFormat Integer.MAX_VALUE JBOD JDBC From dccdfdbff6a122636169610375dd467ccb9d8722 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 20 Nov 2019 23:56:48 -0800 Subject: [PATCH 07/11] refactor toMap conversion so can be shared through flattenerMaker, default impls should be good enough for orc+avro, fixup for merge with latest --- .../org/apache/druid/data/input/Firehose.java | 4 +- .../common/parsers/JSONFlattenerMaker.java | 11 ++- .../util/common/parsers/ObjectFlattener.java | 2 + .../util/common/parsers/ObjectFlatteners.java | 75 +++++++++++++++++++ .../data/input/avro/AvroFlattenerMaker.java | 10 ++- .../input/orc/OrcStructFlattenerMaker.java | 11 ++- .../parquet/ParquetExtensionsModule.java | 5 +- ...putFormat.java => ParquetInputFormat.java} | 11 +-- ...dParquetReader.java => ParquetReader.java} | 62 +++------------ .../parquet/simple/ParquetGroupConverter.java | 16 ---- .../simple/ParquetGroupFlattenerMaker.java | 31 +++++++- .../input/parquet/BaseParquetReaderTest.java | 14 ++-- .../parquet/CompatParquetReaderTest.java | 35 +++++---- .../parquet/DecimalParquetReaderTest.java | 18 ++--- .../parquet/FlattenSpecParquetReaderTest.java | 36 ++++----- .../parquet/TimestampsParquetReaderTest.java | 20 ++--- .../input/parquet/WikiParquetReaderTest.java | 8 +- 17 files changed, 223 insertions(+), 146 deletions(-) rename extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/{DruidNativeParquetInputFormat.java => ParquetInputFormat.java} (86%) rename extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/{DruidParquetReader.java => ParquetReader.java} (62%) diff --git a/core/src/main/java/org/apache/druid/data/input/Firehose.java b/core/src/main/java/org/apache/druid/data/input/Firehose.java index 6c7605ca11d0..66995bda184a 100644 --- a/core/src/main/java/org/apache/druid/data/input/Firehose.java +++ b/core/src/main/java/org/apache/druid/data/input/Firehose.java @@ -69,11 +69,11 @@ public interface Firehose extends Closeable /** * Returns an {@link InputRowListPlusRawValues} object containing the InputRow plus the raw, unparsed data corresponding to * the next row available. Used in the sampler to provide the caller with information to assist in configuring a parse - * spec. If a ParseException is thrown by the parser, it should be caught and returned in the InputRowListPlusJson so + * spec. If a ParseException is thrown by the parser, it should be caught and returned in the InputRowListPlusRawValues so * we will be able to provide information on the raw row which failed to be parsed. Should only be called if hasMore * returns true. * - * @return an InputRowListPlusJson which may contain any of: an InputRow, map of the raw data, or a ParseException + * @return an InputRowListPlusRawValues which may contain any of: an InputRow, map of the raw data, or a ParseException */ @Deprecated default InputRowListPlusRawValues nextRowWithRaw() throws IOException diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java index 8d58451f34bb..e0241c97bc19 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java @@ -24,6 +24,7 @@ import com.jayway.jsonpath.Configuration; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.Option; +import com.jayway.jsonpath.spi.json.JsonProvider; import com.jayway.jsonpath.spi.mapper.JacksonMappingProvider; import net.thisptr.jackson.jq.JsonQuery; import net.thisptr.jackson.jq.exception.JsonQueryException; @@ -43,9 +44,11 @@ public class JSONFlattenerMaker implements ObjectFlatteners.FlattenerMaker { + private static final JsonProvider JSON_PROVIDER = new FastJacksonJsonNodeJsonProvider(); + private static final Configuration JSONPATH_CONFIGURATION = Configuration.builder() - .jsonProvider(new FastJacksonJsonNodeJsonProvider()) + .jsonProvider(JSON_PROVIDER) .mappingProvider(new JacksonMappingProvider()) .options(EnumSet.of(Option.SUPPRESS_EXCEPTIONS)) .build(); @@ -97,6 +100,12 @@ public Function makeJsonQueryExtractor(final String expr) } } + @Override + public JsonProvider getJsonProvider() + { + return null; + } + @Nullable private Object valueConversionFunction(JsonNode val) { diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlattener.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlattener.java index ae674c84e95e..36e7ca34acc3 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlattener.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlattener.java @@ -24,4 +24,6 @@ public interface ObjectFlattener { Map flatten(T obj); + + Map toMap(T obj); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java index 2cdb2339f06d..b40f226e2702 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java @@ -20,13 +20,17 @@ package org.apache.druid.java.util.common.parsers; import com.google.common.collect.Iterables; +import com.jayway.jsonpath.spi.json.JsonProvider; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.UOE; import java.util.AbstractMap; +import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.LinkedHashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Function; @@ -194,17 +198,88 @@ public Object setValue(final Object value) } }; } + + @Override + public Map toMap(T obj) + { + return flattenerMaker.toMap(obj); + } }; } public interface FlattenerMaker { + JsonProvider getJsonProvider(); + /** + * List all "root" primitive properties and primitive lists (no nested objects, no lists of objects) + */ Iterable discoverRootFields(T obj); + /** + * Get a top level field from a "json" object + */ Object getRootField(T obj, String key); + /** + * Create a "field" extractor for {@link com.jayway.jsonpath.JsonPath} expressions + */ Function makeJsonPathExtractor(String expr); + /** + * Create a "field" extractor for 'jq' expressions + */ Function makeJsonQueryExtractor(String expr); + + /** + * Convert object to Java {@link Map} using {@link #getJsonProvider()} and {@link #finalizeConversionForMap} to + * extract and convert data + */ + default Map toMap(T obj) + { + return (Map) toMapHelper(obj); + } + + /** + * Recursively traverse "json" object using a {@link JsonProvider}, converting to Java {@link Map} and {@link List}, + * potentially transforming via {@link #finalizeConversionForMap} as we go + */ + default Object toMapHelper(Object o) + { + final JsonProvider parquetJsonProvider = getJsonProvider(); + if (parquetJsonProvider.isMap(o)) { + Map actualMap = new HashMap<>(); + for (String key : parquetJsonProvider.getPropertyKeys(o)) { + Object field = parquetJsonProvider.getMapValue(o, key); + if (parquetJsonProvider.isMap(field) || parquetJsonProvider.isArray(field)) { + actualMap.put(key, toMapHelper(finalizeConversionForMap(field))); + } else { + actualMap.put(key, finalizeConversionForMap(field)); + } + } + return actualMap; + } else if (parquetJsonProvider.isArray(o)) { + final int length = parquetJsonProvider.length(o); + List actualList = new ArrayList<>(length); + for (int i = 0; i < length; i++) { + Object element = parquetJsonProvider.getArrayIndex(o, i); + if (parquetJsonProvider.isMap(element) || parquetJsonProvider.isArray(element)) { + actualList.add(toMapHelper(finalizeConversionForMap(element))); + } else { + actualList.add(finalizeConversionForMap(element)); + } + } + return finalizeConversionForMap(actualList); + } + // unknown, just pass it through + return o; + } + + /** + * Handle any special conversions for object when translating an input type into a {@link Map} for {@link #toMap} + */ + default Object finalizeConversionForMap(Object o) + { + return o; + } } } diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java index a142dcab9ee8..1957bf9ae5ad 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java @@ -22,6 +22,7 @@ import com.jayway.jsonpath.Configuration; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.Option; +import com.jayway.jsonpath.spi.json.JsonProvider; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.util.Utf8; @@ -39,9 +40,10 @@ public class AvroFlattenerMaker implements ObjectFlatteners.FlattenerMaker { + private static final JsonProvider AVRO_JSON_PROVIDER = new GenericAvroJsonProvider(); private static final Configuration JSONPATH_CONFIGURATION = Configuration.builder() - .jsonProvider(new GenericAvroJsonProvider()) + .jsonProvider(AVRO_JSON_PROVIDER) .mappingProvider(new NotImplementedMappingProvider()) .options(EnumSet.of(Option.SUPPRESS_EXCEPTIONS)) .build(); @@ -125,6 +127,12 @@ public Function makeJsonQueryExtractor(final String expr) throw new UnsupportedOperationException("Avro + JQ not supported"); } + @Override + public JsonProvider getJsonProvider() + { + return AVRO_JSON_PROVIDER; + } + private Object transformValue(final Object field) { if (field instanceof ByteBuffer) { diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java index a7094666255a..f947a8e0c810 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java @@ -22,6 +22,7 @@ import com.jayway.jsonpath.Configuration; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.Option; +import com.jayway.jsonpath.spi.json.JsonProvider; import org.apache.druid.java.util.common.parsers.NotImplementedMappingProvider; import org.apache.druid.java.util.common.parsers.ObjectFlatteners; import org.apache.orc.TypeDescription; @@ -38,13 +39,15 @@ public class OrcStructFlattenerMaker implements ObjectFlatteners.FlattenerMaker { private final Configuration jsonPathConfiguration; + private final JsonProvider orcJsonProvider; private final OrcStructConverter converter; OrcStructFlattenerMaker(boolean binaryAsString) { this.converter = new OrcStructConverter(binaryAsString); + this.orcJsonProvider = new OrcStructJsonProvider(converter); this.jsonPathConfiguration = Configuration.builder() - .jsonProvider(new OrcStructJsonProvider(converter)) + .jsonProvider(orcJsonProvider) .mappingProvider(new NotImplementedMappingProvider()) .options(EnumSet.of(Option.SUPPRESS_EXCEPTIONS)) .build(); @@ -88,6 +91,12 @@ public Function makeJsonQueryExtractor(String expr) throw new UnsupportedOperationException("ORC flattener does not support JQ"); } + @Override + public JsonProvider getJsonProvider() + { + return orcJsonProvider; + } + private Object finalizeConversion(Object o) { // replace any remaining complex types with null diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java index 3c425d37e1d7..e20067cedf09 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java @@ -61,7 +61,7 @@ public List getJacksonModules() new NamedType(ParquetAvroHadoopInputRowParser.class, PARQUET_AVRO_INPUT_PARSER_TYPE), new NamedType(ParquetHadoopInputRowParser.class, PARQUET_SIMPLE_INPUT_PARSER_TYPE), new NamedType(ParquetParseSpec.class, PARQUET_SIMPLE_INPUT_PARSER_TYPE), - new NamedType(DruidNativeParquetInputFormat.class, PARQUET_SIMPLE_PARSE_SPEC_TYPE) + new NamedType(ParquetInputFormat.class, PARQUET_SIMPLE_PARSE_SPEC_TYPE) ) ); } @@ -69,6 +69,9 @@ public List getJacksonModules() @Override public void configure(Binder binder) { + // this block of code is common among extensions that use Hadoop things but are not running in Hadoop, in order + // to properly initialize everything + final Configuration conf = new Configuration(); // Set explicit CL. Otherwise it'll try to use thread context CL, which may not have all of our dependencies. diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidNativeParquetInputFormat.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetInputFormat.java similarity index 86% rename from extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidNativeParquetInputFormat.java rename to extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetInputFormat.java index bb455918eb9f..fbd93414a4a1 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidNativeParquetInputFormat.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetInputFormat.java @@ -32,15 +32,12 @@ import java.io.IOException; import java.util.Objects; -/** - * heh, DruidParquetInputFormat already exists, so I need another name - */ -public class DruidNativeParquetInputFormat extends NestedInputFormat +public class ParquetInputFormat extends NestedInputFormat { private final boolean binaryAsString; @JsonCreator - public DruidNativeParquetInputFormat( + public ParquetInputFormat( @JsonProperty("flattenSpec") @Nullable JSONPathSpec flattenSpec, @JsonProperty("binaryAsString") @Nullable Boolean binaryAsString ) @@ -68,7 +65,7 @@ public InputEntityReader createReader( File temporaryDirectory ) throws IOException { - return new DruidParquetReader(inputRowSchema, source, temporaryDirectory, getFlattenSpec(), binaryAsString); + return new ParquetReader(inputRowSchema, source, temporaryDirectory, getFlattenSpec(), binaryAsString); } @Override @@ -83,7 +80,7 @@ public boolean equals(Object o) if (!super.equals(o)) { return false; } - DruidNativeParquetInputFormat that = (DruidNativeParquetInputFormat) o; + ParquetInputFormat that = (ParquetInputFormat) o; return binaryAsString == that.binaryAsString; } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java similarity index 62% rename from extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java rename to extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java index 3a0a2b5c796e..627d0f77a1dd 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/DruidParquetReader.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java @@ -24,9 +24,7 @@ import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.IntermediateRowParsingReader; import org.apache.druid.data.input.impl.MapInputRowParser; -import org.apache.druid.data.input.parquet.simple.ParquetGroupConverter; import org.apache.druid.data.input.parquet.simple.ParquetGroupFlattenerMaker; -import org.apache.druid.data.input.parquet.simple.ParquetGroupJsonProvider; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.java.util.common.parsers.JSONPathSpec; @@ -37,33 +35,25 @@ import org.apache.hadoop.fs.Path; import org.apache.parquet.example.data.Group; import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.example.GroupReadSupport; import org.apache.parquet.hadoop.metadata.ParquetMetadata; import java.io.File; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; -public class DruidParquetReader extends IntermediateRowParsingReader +public class ParquetReader extends IntermediateRowParsingReader { private final InputRowSchema inputRowSchema; private final ObjectFlattener flattener; - private final byte[] buffer = new byte[InputEntity.DEFAULT_FETCH_BUFFER_SIZE]; - private final ParquetGroupConverter converter; - private final ParquetGroupJsonProvider jsonProvider; - - private final ParquetReader reader; - private final ParquetMetadata metadata; + private final org.apache.parquet.hadoop.ParquetReader reader; private final Closer closer; - public DruidParquetReader( + public ParquetReader( InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory, @@ -72,19 +62,18 @@ public DruidParquetReader( ) throws IOException { this.inputRowSchema = inputRowSchema; - this.converter = new ParquetGroupConverter(binaryAsString); - this.jsonProvider = new ParquetGroupJsonProvider(converter); this.flattener = ObjectFlatteners.create(flattenSpec, new ParquetGroupFlattenerMaker(binaryAsString)); closer = Closer.create(); + byte[] buffer = new byte[InputEntity.DEFAULT_FETCH_BUFFER_SIZE]; final InputEntity.CleanableFile file = closer.register(source.fetch(temporaryDirectory, buffer)); final Path path = new Path(file.file().toURI()); final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); try { Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); - reader = closer.register(ParquetReader.builder(new GroupReadSupport(), path).build()); - metadata = ParquetFileReader.readFooter(new Configuration(), path); + reader = closer.register(org.apache.parquet.hadoop.ParquetReader.builder(new GroupReadSupport(), path).build()); + ParquetMetadata metadata = ParquetFileReader.readFooter(new Configuration(), path); } finally { Thread.currentThread().setContextClassLoader(currentClassLoader); @@ -92,7 +81,7 @@ public DruidParquetReader( } @Override - protected CloseableIterator intermediateRowIterator() throws IOException + protected CloseableIterator intermediateRowIterator() { return new CloseableIterator() { @@ -132,7 +121,7 @@ public void close() throws IOException } @Override - protected List parseInputRows(Group intermediateRow) throws IOException, ParseException + protected List parseInputRows(Group intermediateRow) throws ParseException { return Collections.singletonList( MapInputRowParser.parse( @@ -144,39 +133,8 @@ protected List parseInputRows(Group intermediateRow) throws IOExceptio } @Override - protected String toJson(Group intermediateRow) throws IOException + protected Map toMap(Group intermediateRow) { - Object converted = convertObject(intermediateRow); - return DEFAULT_JSON_WRITER.writeValueAsString(converted); - } - - private Object convertObject(Object o) - { - if (jsonProvider.isMap(o)) { - Map actualMap = new HashMap<>(); - for (String key : jsonProvider.getPropertyKeys(o)) { - Object field = jsonProvider.getMapValue(o, key); - if (jsonProvider.isMap(field) || jsonProvider.isArray(field)) { - actualMap.put(key, convertObject(converter.finalizeConversion(field))); - } else { - actualMap.put(key, converter.finalizeConversion(field)); - } - } - return actualMap; - } else if (jsonProvider.isArray(o)) { - final int length = jsonProvider.length(o); - List actualList = new ArrayList<>(length); - for (int i = 0; i < length; i++) { - Object element = jsonProvider.getArrayIndex(o, i); - if (jsonProvider.isMap(element) || jsonProvider.isArray(element)) { - actualList.add(convertObject(converter.finalizeConversion(element))); - } else { - actualList.add(converter.finalizeConversion(element)); - } - } - return converter.finalizeConversion(actualList); - } - // unknown, just pass it through - return o; + return flattener.toMap(intermediateRow); } } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java index 78898c09f0a8..0a076ff0ef4a 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java @@ -42,9 +42,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; public class ParquetGroupConverter { @@ -500,18 +498,4 @@ public Object unwrapListPrimitive(Object o) Group g = (Group) o; return convertPrimitiveField(g, 0, binaryAsString); } - - public Object finalizeConversion(Object o) - { - // conversion can leave 'wrapped' list primitives - if (isWrappedListPrimitive(o)) { - return unwrapListPrimitive(o); - } else if (o instanceof List) { - List asList = ((List) o).stream().filter(Objects::nonNull).collect(Collectors.toList()); - if (asList.stream().allMatch(ParquetGroupConverter::isWrappedListPrimitive)) { - return asList.stream().map(Group.class::cast).map(this::unwrapListPrimitive).collect(Collectors.toList()); - } - } - return o; - } } diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java index b091e172aa49..b7681c2dd873 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupFlattenerMaker.java @@ -22,6 +22,7 @@ import com.jayway.jsonpath.Configuration; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.Option; +import com.jayway.jsonpath.spi.json.JsonProvider; import org.apache.druid.java.util.common.parsers.NotImplementedMappingProvider; import org.apache.druid.java.util.common.parsers.ObjectFlatteners; import org.apache.parquet.example.data.Group; @@ -29,21 +30,24 @@ import javax.annotation.Nullable; import java.util.EnumSet; +import java.util.List; +import java.util.Objects; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; public class ParquetGroupFlattenerMaker implements ObjectFlatteners.FlattenerMaker { - private final Configuration jsonPathConfiguration; private final ParquetGroupConverter converter; + private final JsonProvider parquetJsonProvider; public ParquetGroupFlattenerMaker(boolean binaryAsString) { this.converter = new ParquetGroupConverter(binaryAsString); + this.parquetJsonProvider = new ParquetGroupJsonProvider(converter); this.jsonPathConfiguration = Configuration.builder() - .jsonProvider(new ParquetGroupJsonProvider(converter)) + .jsonProvider(parquetJsonProvider) .mappingProvider(new NotImplementedMappingProvider()) .options(EnumSet.of(Option.SUPPRESS_EXCEPTIONS)) .build(); @@ -84,6 +88,18 @@ public Function makeJsonQueryExtractor(String expr) throw new UnsupportedOperationException("Parquet does not support JQ"); } + @Override + public JsonProvider getJsonProvider() + { + return parquetJsonProvider; + } + + @Override + public Object finalizeConversionForMap(Object o) + { + return finalizeConversion(o); + } + /** * After json conversion, wrapped list items can still need unwrapped. See * {@link ParquetGroupConverter#isWrappedListPrimitive(Object)} and @@ -95,6 +111,15 @@ public Function makeJsonQueryExtractor(String expr) */ private Object finalizeConversion(Object o) { - return converter.finalizeConversion(o); + // conversion can leave 'wrapped' list primitives + if (ParquetGroupConverter.isWrappedListPrimitive(o)) { + return converter.unwrapListPrimitive(o); + } else if (o instanceof List) { + List asList = ((List) o).stream().filter(Objects::nonNull).collect(Collectors.toList()); + if (asList.stream().allMatch(ParquetGroupConverter::isWrappedListPrimitive)) { + return asList.stream().map(Group.class::cast).map(converter::unwrapListPrimitive).collect(Collectors.toList()); + } + } + return o; } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java index 69010a198170..fecde6cc1ff6 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java @@ -19,9 +19,11 @@ package org.apache.druid.data.input.parquet; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.FileEntity; import org.apache.druid.java.util.common.parsers.CloseableIterator; @@ -34,6 +36,8 @@ public class BaseParquetReaderTest { + ObjectWriter DEFAULT_JSON_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter(); + InputEntityReader createReader(String parquetFile, InputRowSchema schema, JSONPathSpec flattenSpec) throws IOException { return createReader(parquetFile, schema, flattenSpec, false); @@ -47,7 +51,7 @@ InputEntityReader createReader( ) throws IOException { FileEntity entity = new FileEntity(new File(parquetFile)); - DruidNativeParquetInputFormat parquet = new DruidNativeParquetInputFormat(flattenSpec, binaryAsString); + ParquetInputFormat parquet = new ParquetInputFormat(flattenSpec, binaryAsString); InputEntityReader reader = parquet.createReader(schema, entity, null); return reader; } @@ -63,10 +67,10 @@ List readAllRows(InputEntityReader reader) throws IOException return rows; } - List sampleAllRows(InputEntityReader reader) throws IOException + List sampleAllRows(InputEntityReader reader) throws IOException { - List rows = new ArrayList<>(); - try (CloseableIterator iterator = reader.sample()) { + List rows = new ArrayList<>(); + try (CloseableIterator iterator = reader.sample()) { while (iterator.hasNext()) { rows.add(iterator.next()); } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java index f12f3373770f..f8b586bec67d 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/CompatParquetReaderTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -38,7 +38,7 @@ import java.util.List; /** - * Duplicate of {@link CompatParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + * Duplicate of {@link CompatParquetInputTest} but for {@link ParquetReader} instead of Hadoop */ public class CompatParquetReaderTest extends BaseParquetReaderTest { @@ -88,19 +88,22 @@ public void testBinaryAsString() throws IOException JSONPathSpec.DEFAULT, false ); - List sampled = sampleAllRows(reader); - List sampledAsBinary = sampleAllRows(readerNotAsString); + List sampled = sampleAllRows(reader); + List sampledAsBinary = sampleAllRows(readerNotAsString); final String expectedJson = "{\n" + " \"field\" : \"hey this is &é(-è_çà)=^$ù*! Ω^^\",\n" + " \"ts\" : 1471800234\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); final String expectedJsonBinary = "{\n" + " \"field\" : \"aGV5IHRoaXMgaXMgJsOpKC3DqF/Dp8OgKT1eJMO5KiEgzqleXg==\",\n" + " \"ts\" : 1471800234\n" + "}"; - Assert.assertEquals(expectedJsonBinary, sampledAsBinary.get(0).getRawJson()); + Assert.assertEquals( + expectedJsonBinary, + DEFAULT_JSON_WRITER.writeValueAsString(sampledAsBinary.get(0).getRawValues()) + ); } @@ -136,11 +139,11 @@ public void testParquet1217() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"col\" : -1\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -242,7 +245,7 @@ required group nestedIntsColumn (LIST) { schema, flattenSpec ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"enumColumn\" : \"SPADES\",\n" + " \"maybeStringColumn\" : { },\n" @@ -302,7 +305,7 @@ required group nestedIntsColumn (LIST) { + " } ]\n" + " }\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -335,11 +338,11 @@ public void testOldRepeatedInt() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"repeatedInt\" : [ 1, 2, 3 ]\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @@ -374,7 +377,7 @@ public void testReadNestedArrayStruct() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"primitive\" : 2,\n" + " \"myComplex\" : [ {\n" @@ -382,7 +385,7 @@ public void testReadNestedArrayStruct() throws IOException + " \"repeatedMessage\" : [ 3 ]\n" + " } ]\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -421,7 +424,7 @@ public void testProtoStructWithArray() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"optionalMessage\" : { },\n" + " \"requiredPrimitive\" : 9,\n" @@ -432,6 +435,6 @@ public void testProtoStructWithArray() throws IOException + " \"someId\" : 9\n" + " }\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java index 5511eb7f677d..50b9fe2df27d 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/DecimalParquetReaderTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -38,7 +38,7 @@ import java.util.List; /** - * Duplicate of {@link DecimalParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + * Duplicate of {@link DecimalParquetInputTest} but for {@link ParquetReader} instead of Hadoop */ public class DecimalParquetReaderTest extends BaseParquetReaderTest { @@ -72,11 +72,11 @@ public void testReadParquetDecimalFixedLen() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"fixed_len_dec\" : 1.0\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(1).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(1).getRawValues())); } @Test @@ -109,15 +109,15 @@ public void testReadParquetDecimali32() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"i32_dec\" : 100\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(1).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(1).getRawValues())); } @Test - public void testReadParquetDecimali64() throws IOException, InterruptedException + public void testReadParquetDecimali64() throws IOException { final String file = "example/decimals/dec-in-i64.parquet"; InputRowSchema schema = new InputRowSchema( @@ -146,10 +146,10 @@ public void testReadParquetDecimali64() throws IOException, InterruptedException schema, flattenSpec ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"i64_dec\" : 100\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(1).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(1).getRawValues())); } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java index 4a7432aa2cf6..5be38dda494d 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/FlattenSpecParquetReaderTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -37,7 +37,7 @@ import java.util.List; /** - * Duplicate of {@link FlattenSpecParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + * Duplicate of {@link FlattenSpecParquetInputTest} but for {@link ParquetReader} instead of Hadoop */ public class FlattenSpecParquetReaderTest extends BaseParquetReaderTest { @@ -92,8 +92,8 @@ public void testFlat1NoFlattenSpec() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); - Assert.assertEquals(FLAT_JSON, sampled.get(0).getRawJson()); + List sampled = sampleAllRows(reader); + Assert.assertEquals(FLAT_JSON, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -125,8 +125,8 @@ public void testFlat1Autodiscover() throws IOException schema, JSONPathSpec.DEFAULT ); - List sampled = sampleAllRows(reader); - Assert.assertEquals(FLAT_JSON, sampled.get(0).getRawJson()); + List sampled = sampleAllRows(reader); + Assert.assertEquals(FLAT_JSON, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -166,8 +166,8 @@ public void testFlat1Flatten() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); - Assert.assertEquals(FLAT_JSON, sampled.get(0).getRawJson()); + List sampled = sampleAllRows(reader); + Assert.assertEquals(FLAT_JSON, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -204,9 +204,9 @@ public void testFlat1FlattenSelectListItem() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); - Assert.assertEquals(FLAT_JSON, sampled.get(0).getRawJson()); + Assert.assertEquals(FLAT_JSON, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @@ -242,8 +242,8 @@ public void testNested1NoFlattenSpec() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); - Assert.assertEquals(NESTED_JSON, sampled.get(0).getRawJson()); + List sampled = sampleAllRows(reader); + Assert.assertEquals(NESTED_JSON, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -275,8 +275,8 @@ public void testNested1Autodiscover() throws IOException schema, JSONPathSpec.DEFAULT ); - List sampled = sampleAllRows(reader); - Assert.assertEquals(NESTED_JSON, sampled.get(0).getRawJson()); + List sampled = sampleAllRows(reader); + Assert.assertEquals(NESTED_JSON, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -318,8 +318,8 @@ public void testNested1Flatten() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); - Assert.assertEquals(NESTED_JSON, sampled.get(0).getRawJson()); + List sampled = sampleAllRows(reader); + Assert.assertEquals(NESTED_JSON, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -359,7 +359,7 @@ public void testNested1FlattenSelectListItem() throws IOException schema, flattenSpec ); - List sampled = sampleAllRows(reader); - Assert.assertEquals(NESTED_JSON, sampled.get(0).getRawJson()); + List sampled = sampleAllRows(reader); + Assert.assertEquals(NESTED_JSON, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java index 6ee4173fc140..19f1544dcff0 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/TimestampsParquetReaderTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -35,7 +35,7 @@ import java.util.List; /** - * Duplicate of {@link TimestampsParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + * Duplicate of {@link TimestampsParquetInputTest} but for {@link ParquetReader} instead of Hadoop */ public class TimestampsParquetReaderTest extends BaseParquetReaderTest { @@ -82,8 +82,8 @@ public void testDateHandling() throws IOException schemaAsDate, JSONPathSpec.DEFAULT ); - List sampledAsString = sampleAllRows(readerAsString); - List sampledAsDate = sampleAllRows(readerAsDate); + List sampledAsString = sampleAllRows(readerAsString); + List sampledAsDate = sampleAllRows(readerAsDate); final String expectedJson = "{\n" + " \"date_as_string\" : \"2017-06-18\",\n" + " \"timestamp_as_timestamp\" : 1497702471815,\n" @@ -91,8 +91,8 @@ public void testDateHandling() throws IOException + " \"idx\" : 1,\n" + " \"date_as_date\" : 1497744000000\n" + "}"; - Assert.assertEquals(expectedJson, sampledAsString.get(0).getRawJson()); - Assert.assertEquals(expectedJson, sampledAsDate.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampledAsString.get(0).getRawValues())); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampledAsDate.get(0).getRawValues())); } @Test @@ -116,11 +116,11 @@ public void testParseInt96Timestamp() throws IOException schema, JSONPathSpec.DEFAULT ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"ts\" : 978310861000\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } @Test @@ -146,10 +146,10 @@ public void testTimeMillisInInt64() throws IOException schema, JSONPathSpec.DEFAULT ); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"time\" : 10\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java index b2c1977146ef..75e5e916ec78 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/WikiParquetReaderTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.InputEntityReader; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.InputRowListPlusJson; +import org.apache.druid.data.input.InputRowListPlusRawValues; import org.apache.druid.data.input.InputRowSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; @@ -35,7 +35,7 @@ import java.util.List; /** - * Duplicate of {@link WikiParquetInputTest} but for {@link DruidParquetReader} instead of Hadoop + * Duplicate of {@link WikiParquetInputTest} but for {@link ParquetReader} instead of Hadoop */ public class WikiParquetReaderTest extends BaseParquetReaderTest { @@ -57,7 +57,7 @@ public void testWiki() throws IOException Assert.assertEquals("zh", s2); reader = createReader("example/wiki/wiki.parquet", schema, JSONPathSpec.DEFAULT); - List sampled = sampleAllRows(reader); + List sampled = sampleAllRows(reader); final String expectedJson = "{\n" + " \"continent\" : \"North America\",\n" @@ -77,6 +77,6 @@ public void testWiki() throws IOException + " \"user\" : \"nuclear\",\n" + " \"timestamp\" : \"2013-08-31T01:02:33Z\"\n" + "}"; - Assert.assertEquals(expectedJson, sampled.get(0).getRawJson()); + Assert.assertEquals(expectedJson, DEFAULT_JSON_WRITER.writeValueAsString(sampled.get(0).getRawValues())); } } From 1002cb8f65525ccb8477d8aab3709cb5c22937ea Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 21 Nov 2019 00:32:24 -0800 Subject: [PATCH 08/11] add comment, fix some stuff --- .../util/common/parsers/ObjectFlatteners.java | 18 +++++++++--------- extensions-core/parquet-extensions/pom.xml | 9 ++++++++- .../data/input/parquet/ParquetReader.java | 4 ---- 3 files changed, 17 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java index b40f226e2702..2130c016d056 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/ObjectFlatteners.java @@ -245,24 +245,24 @@ default Map toMap(T obj) */ default Object toMapHelper(Object o) { - final JsonProvider parquetJsonProvider = getJsonProvider(); - if (parquetJsonProvider.isMap(o)) { + final JsonProvider jsonProvider = getJsonProvider(); + if (jsonProvider.isMap(o)) { Map actualMap = new HashMap<>(); - for (String key : parquetJsonProvider.getPropertyKeys(o)) { - Object field = parquetJsonProvider.getMapValue(o, key); - if (parquetJsonProvider.isMap(field) || parquetJsonProvider.isArray(field)) { + for (String key : jsonProvider.getPropertyKeys(o)) { + Object field = jsonProvider.getMapValue(o, key); + if (jsonProvider.isMap(field) || jsonProvider.isArray(field)) { actualMap.put(key, toMapHelper(finalizeConversionForMap(field))); } else { actualMap.put(key, finalizeConversionForMap(field)); } } return actualMap; - } else if (parquetJsonProvider.isArray(o)) { - final int length = parquetJsonProvider.length(o); + } else if (jsonProvider.isArray(o)) { + final int length = jsonProvider.length(o); List actualList = new ArrayList<>(length); for (int i = 0; i < length; i++) { - Object element = parquetJsonProvider.getArrayIndex(o, i); - if (parquetJsonProvider.isMap(element) || parquetJsonProvider.isArray(element)) { + Object element = jsonProvider.getArrayIndex(o, i); + if (jsonProvider.isMap(element) || jsonProvider.isArray(element)) { actualList.add(toMapHelper(finalizeConversionForMap(element))); } else { actualList.add(finalizeConversionForMap(element)); diff --git a/extensions-core/parquet-extensions/pom.xml b/extensions-core/parquet-extensions/pom.xml index 392699e3a84c..b45805902576 100644 --- a/extensions-core/parquet-extensions/pom.xml +++ b/extensions-core/parquet-extensions/pom.xml @@ -137,6 +137,14 @@ ${project.parent.version} provided + org.apache.hadoop hadoop-mapreduce-client-core @@ -202,7 +210,6 @@ hadoop-common ${hadoop.compile.version} compile - org.apache.yetus diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java index 627d0f77a1dd..492473df10a9 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java @@ -31,12 +31,9 @@ import org.apache.druid.java.util.common.parsers.ObjectFlattener; import org.apache.druid.java.util.common.parsers.ObjectFlatteners; import org.apache.druid.java.util.common.parsers.ParseException; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.parquet.example.data.Group; -import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.example.GroupReadSupport; -import org.apache.parquet.hadoop.metadata.ParquetMetadata; import java.io.File; import java.io.IOException; @@ -73,7 +70,6 @@ public ParquetReader( try { Thread.currentThread().setContextClassLoader(getClass().getClassLoader()); reader = closer.register(org.apache.parquet.hadoop.ParquetReader.builder(new GroupReadSupport(), path).build()); - ParquetMetadata metadata = ParquetFileReader.readFooter(new Configuration(), path); } finally { Thread.currentThread().setContextClassLoader(currentClassLoader); From 3fd476429d8e2a62d4ee832ec8c553e052142774 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 21 Nov 2019 03:38:48 -0800 Subject: [PATCH 09/11] adjustments --- .../druid/data/input/parquet/ParquetExtensionsModule.java | 3 +-- .../apache/druid/data/input/parquet/BaseParquetReaderTest.java | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java index e20067cedf09..dd2cf4c34f82 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetExtensionsModule.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.inject.Binder; import com.google.inject.Inject; -import com.google.inject.TypeLiteral; import org.apache.druid.data.input.parquet.avro.ParquetAvroHadoopInputRowParser; import org.apache.druid.data.input.parquet.simple.ParquetHadoopInputRowParser; import org.apache.druid.data.input.parquet.simple.ParquetParseSpec; @@ -99,6 +98,6 @@ public void configure(Binder binder) } } - binder.requestInjection(TypeLiteral.get(Configuration.class), conf); + binder.bind(Configuration.class).toInstance(conf); } } diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java index fecde6cc1ff6..5c19b0e3d731 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java @@ -34,7 +34,7 @@ import java.util.ArrayList; import java.util.List; -public class BaseParquetReaderTest +class BaseParquetReaderTest { ObjectWriter DEFAULT_JSON_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter(); From addb9d4713b129df60c3289ca6e88d004f0589b9 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 21 Nov 2019 05:10:26 -0800 Subject: [PATCH 10/11] fix accident --- .../druid/java/util/common/parsers/JSONFlattenerMaker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java index e0241c97bc19..135c70271c23 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/JSONFlattenerMaker.java @@ -103,7 +103,7 @@ public Function makeJsonQueryExtractor(final String expr) @Override public JsonProvider getJsonProvider() { - return null; + return JSON_PROVIDER; } @Nullable From be9f745435294f386bbf69a5f990dfce3de054ff Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 21 Nov 2019 18:53:53 -0800 Subject: [PATCH 11/11] tweaks --- .../druid/data/input/parquet/ParquetReader.java | 2 +- .../parquet/simple/ParquetGroupConverter.java | 8 ++++---- .../data/input/parquet/BaseParquetInputTest.java | 14 +++++++------- .../data/input/parquet/BaseParquetReaderTest.java | 11 +++-------- 4 files changed, 15 insertions(+), 20 deletions(-) diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java index 492473df10a9..a98f929a1c03 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/ParquetReader.java @@ -50,7 +50,7 @@ public class ParquetReader extends IntermediateRowParsingReader private final org.apache.parquet.hadoop.ParquetReader reader; private final Closer closer; - public ParquetReader( + ParquetReader( InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory, diff --git a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java index 0a076ff0ef4a..24d5f556475c 100644 --- a/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java +++ b/extensions-core/parquet-extensions/src/main/java/org/apache/druid/data/input/parquet/simple/ParquetGroupConverter.java @@ -44,7 +44,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; -public class ParquetGroupConverter +class ParquetGroupConverter { private static final int JULIAN_EPOCH_OFFSET_DAYS = 2_440_588; private static final long MILLIS_IN_DAY = TimeUnit.DAYS.toMillis(1); @@ -469,7 +469,7 @@ static boolean isWrappedListPrimitive(Object o) private final boolean binaryAsString; - public ParquetGroupConverter(boolean binaryAsString) + ParquetGroupConverter(boolean binaryAsString) { this.binaryAsString = binaryAsString; } @@ -482,7 +482,7 @@ public ParquetGroupConverter(boolean binaryAsString) * if a field is not present, this method will return null. */ @Nullable - public Object convertField(Group g, String fieldName) + Object convertField(Group g, String fieldName) { return convertField(g, fieldName, binaryAsString); } @@ -492,7 +492,7 @@ public Object convertField(Group g, String fieldName) * return lists which contain 'wrapped' primitives, that are a {@link Group} with a single, primitive field (see * {@link ParquetGroupConverter#isWrappedListPrimitive(Object)}) */ - public Object unwrapListPrimitive(Object o) + Object unwrapListPrimitive(Object o) { assert isWrappedListPrimitive(o); Group g = (Group) o; diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetInputTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetInputTest.java index 3a0da9f89e93..9cfbf0d12306 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetInputTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetInputTest.java @@ -47,21 +47,21 @@ class BaseParquetInputTest { - private static Map parseSpecType = ImmutableMap.of( + private static final Map PARSE_SPEC_TYPES = ImmutableMap.of( ParquetExtensionsModule.PARQUET_AVRO_INPUT_PARSER_TYPE, ParquetExtensionsModule.PARQUET_AVRO_PARSE_SPEC_TYPE, ParquetExtensionsModule.PARQUET_SIMPLE_INPUT_PARSER_TYPE, ParquetExtensionsModule.PARQUET_SIMPLE_PARSE_SPEC_TYPE ); - private static Map inputFormatType = ImmutableMap.of( + private static final Map INPUT_FORMAT_TYPES = ImmutableMap.of( ParquetExtensionsModule.PARQUET_AVRO_INPUT_PARSER_TYPE, "org.apache.druid.data.input.parquet.DruidParquetAvroInputFormat", ParquetExtensionsModule.PARQUET_SIMPLE_INPUT_PARSER_TYPE, "org.apache.druid.data.input.parquet.DruidParquetInputFormat" ); - private static Map> inputFormatClass = ImmutableMap.of( + private static final Map> INPUT_FORMAT_CLASSES = ImmutableMap.of( ParquetExtensionsModule.PARQUET_AVRO_INPUT_PARSER_TYPE, DruidParquetAvroInputFormat.class, ParquetExtensionsModule.PARQUET_SIMPLE_INPUT_PARSER_TYPE, @@ -78,9 +78,9 @@ static HadoopDruidIndexerConfig transformHadoopDruidIndexerConfig( String template = Strings.utf8ToString(Files.readAllBytes(Paths.get(templateFile))); String transformed; if (withParseType) { - transformed = StringUtils.format(template, inputFormatType.get(type), type, parseSpecType.get(type)); + transformed = StringUtils.format(template, INPUT_FORMAT_TYPES.get(type), type, PARSE_SPEC_TYPES.get(type)); } else { - transformed = StringUtils.format(template, inputFormatType.get(type), type); + transformed = StringUtils.format(template, INPUT_FORMAT_TYPES.get(type), type); } return HadoopDruidIndexerConfig.fromString(transformed); } @@ -93,7 +93,7 @@ static Object getFirstRow(Job job, String parserType, String parquetPath) throws FileSplit split = new FileSplit(path, 0, testFile.length(), null); InputFormat inputFormat = ReflectionUtils.newInstance( - inputFormatClass.get(parserType), + INPUT_FORMAT_CLASSES.get(parserType), job.getConfiguration() ); TaskAttemptContext context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID()); @@ -117,7 +117,7 @@ static List getAllRows(String parserType, HadoopDruidIndexerConfig con FileSplit split = new FileSplit(path, 0, testFile.length(), null); InputFormat inputFormat = ReflectionUtils.newInstance( - inputFormatClass.get(parserType), + INPUT_FORMAT_CLASSES.get(parserType), job.getConfiguration() ); TaskAttemptContext context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID()); diff --git a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java index 5c19b0e3d731..ac22f77f5f49 100644 --- a/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java +++ b/extensions-core/parquet-extensions/src/test/java/org/apache/druid/data/input/parquet/BaseParquetReaderTest.java @@ -52,17 +52,14 @@ InputEntityReader createReader( { FileEntity entity = new FileEntity(new File(parquetFile)); ParquetInputFormat parquet = new ParquetInputFormat(flattenSpec, binaryAsString); - InputEntityReader reader = parquet.createReader(schema, entity, null); - return reader; + return parquet.createReader(schema, entity, null); } List readAllRows(InputEntityReader reader) throws IOException { List rows = new ArrayList<>(); try (CloseableIterator iterator = reader.read()) { - while (iterator.hasNext()) { - rows.add(iterator.next()); - } + iterator.forEachRemaining(rows::add); } return rows; } @@ -71,9 +68,7 @@ List sampleAllRows(InputEntityReader reader) throws I { List rows = new ArrayList<>(); try (CloseableIterator iterator = reader.sample()) { - while (iterator.hasNext()) { - rows.add(iterator.next()); - } + iterator.forEachRemaining(rows::add); } return rows; }