jihoonson commented on a change in pull request #9671:
URL: https://github.com/apache/druid/pull/9671#discussion_r421841088
##########
File path: docs/ingestion/data-formats.md
##########
@@ -223,6 +223,41 @@ The Parquet `inputFormat` has the following components:
|flattenSpec| JSON Object |Define a [`flattenSpec`](#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) |
+### Avro OCF
+
+> You need to include the
[`druid-avro-extensions`](../development/extensions-core/avro.md) as an
extension to use the Avro OCF input format.
+
+The `inputFormat` to load data of Avro OCF format. An example is:
+```json
+"ioConfig": {
+ "inputFormat": {
+ "type": "avro_ocf",
+ "flattenSpec": {
+ "useFieldDiscovery": true,
+ "fields": [
+ {
+ "type": "path",
+ "name": "nested",
+ "expr": "$.path.to.nested"
+ }
+ ]
+ },
+ "schema": {
+ ...
Review comment:
Can you add a valid example for `schema` as well? It will help readers
getting some idea about what it would be for them.
##########
File path:
extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFInputFormat.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.avro;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.avro.Schema;
+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.guice.annotations.Json;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.JSONPathSpec;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.util.Map;
+import java.util.Objects;
+
+public class AvroOCFInputFormat extends NestedInputFormat
+{
+ private static final Logger LOGGER = new Logger(AvroOCFInputFormat.class);
+
+ private final boolean binaryAsString;
+ private final Schema readerSchema;
Review comment:
Please add `@Nullable`.
##########
File path:
extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+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.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 java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class AvroOCFReader extends IntermediateRowParsingReader<GenericRecord>
+{
+ private final InputRowSchema inputRowSchema;
+ private final InputEntity source;
+ private final File temporaryDirectory;
+ private final ObjectFlattener<GenericRecord> recordFlattener;
+ private Schema readerSchema;
+
+ AvroOCFReader(
+ InputRowSchema inputRowSchema,
+ InputEntity source,
+ File temporaryDirectory,
+ Schema readerSchema,
+ JSONPathSpec flattenSpec,
+ boolean binaryAsString
+ )
+ {
+ this.inputRowSchema = inputRowSchema;
+ this.source = source;
+ this.temporaryDirectory = temporaryDirectory;
+ this.readerSchema = readerSchema;
+ this.recordFlattener = ObjectFlatteners.create(flattenSpec, new
AvroFlattenerMaker(false, binaryAsString));
+ }
+
+ private static Schema dataFileSchema(File file) throws IOException
+ {
+ return new DataFileReader<GenericRecord>(file, new
GenericDatumReader<>()).getSchema();
Review comment:
The created reader should be closed.
##########
File path:
extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFInputFormat.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.avro;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.avro.Schema;
+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.guice.annotations.Json;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.JSONPathSpec;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.util.Map;
+import java.util.Objects;
+
+public class AvroOCFInputFormat extends NestedInputFormat
+{
+ private static final Logger LOGGER = new Logger(AvroOCFInputFormat.class);
+
+ private final boolean binaryAsString;
+ private final Schema readerSchema;
+
+ @JsonCreator
+ public AvroOCFInputFormat(
+ @JacksonInject @Json ObjectMapper mapper,
+ @JsonProperty("flattenSpec") @Nullable JSONPathSpec flattenSpec,
+ @JsonProperty("schema") Map<String, Object> schema,
Review comment:
Please add `@Nullable`.
##########
File path:
extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+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.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 java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class AvroOCFReader extends IntermediateRowParsingReader<GenericRecord>
+{
+ private final InputRowSchema inputRowSchema;
+ private final InputEntity source;
+ private final File temporaryDirectory;
+ private final ObjectFlattener<GenericRecord> recordFlattener;
+ private Schema readerSchema;
+
+ AvroOCFReader(
+ InputRowSchema inputRowSchema,
+ InputEntity source,
+ File temporaryDirectory,
+ Schema readerSchema,
Review comment:
Please add `@Nullable`.
##########
File path:
extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFInputFormat.java
##########
@@ -0,0 +1,108 @@
+/*
+ * 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.avro;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.avro.Schema;
+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.guice.annotations.Json;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.common.parsers.JSONPathSpec;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.util.Map;
+import java.util.Objects;
+
+public class AvroOCFInputFormat extends NestedInputFormat
+{
+ private static final Logger LOGGER = new Logger(AvroOCFInputFormat.class);
+
+ private final boolean binaryAsString;
+ private final Schema readerSchema;
+
+ @JsonCreator
+ public AvroOCFInputFormat(
+ @JacksonInject @Json ObjectMapper mapper,
+ @JsonProperty("flattenSpec") @Nullable JSONPathSpec flattenSpec,
+ @JsonProperty("schema") Map<String, Object> schema,
+ @JsonProperty("binaryAsString") @Nullable Boolean binaryAsString
+ ) throws Exception
+ {
+ super(flattenSpec);
+ // If a reader schema is supplied create the datum reader with said
schema, otherwise use the writer schema
+ if (schema != null) {
+ String schemaStr = mapper.writeValueAsString(schema);
+ LOGGER.debug("Initialising with reader schema: [%s]", schemaStr);
+ this.readerSchema = new Schema.Parser().parse(schemaStr);
+ } else {
+ this.readerSchema = null;
+ }
+ this.binaryAsString = binaryAsString == null ? false : binaryAsString;
+ }
+
+ @Override
+ public boolean isSplittable()
+ {
+ return false;
Review comment:
This seems fine for now, but we would probably want to make it
splittable in the future. Would you please add a comment about it with this
[link](https://avro.apache.org/docs/current/spec.html#Object+Container+Files)?
##########
File path:
extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroOCFReader.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.avro;
+
+import org.apache.avro.Schema;
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+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.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 java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class AvroOCFReader extends IntermediateRowParsingReader<GenericRecord>
+{
+ private final InputRowSchema inputRowSchema;
+ private final InputEntity source;
+ private final File temporaryDirectory;
+ private final ObjectFlattener<GenericRecord> recordFlattener;
+ private Schema readerSchema;
+
+ AvroOCFReader(
+ InputRowSchema inputRowSchema,
+ InputEntity source,
+ File temporaryDirectory,
+ Schema readerSchema,
+ JSONPathSpec flattenSpec,
+ boolean binaryAsString
+ )
+ {
+ this.inputRowSchema = inputRowSchema;
+ this.source = source;
+ this.temporaryDirectory = temporaryDirectory;
+ this.readerSchema = readerSchema;
+ this.recordFlattener = ObjectFlatteners.create(flattenSpec, new
AvroFlattenerMaker(false, binaryAsString));
+ }
+
+ private static Schema dataFileSchema(File file) throws IOException
+ {
+ return new DataFileReader<GenericRecord>(file, new
GenericDatumReader<>()).getSchema();
+ }
+
+ @Override
+ protected CloseableIterator<GenericRecord> intermediateRowIterator() throws
IOException
+ {
+ final Closer closer = Closer.create();
+
+ final byte[] buffer = new byte[InputEntity.DEFAULT_FETCH_BUFFER_SIZE];
+ final InputEntity.CleanableFile file =
closer.register(source.fetch(temporaryDirectory, buffer));
+ final Schema writerSchema = dataFileSchema(file.file());
+ if (readerSchema == null) {
+ readerSchema = writerSchema;
+ }
+ final GenericDatumReader<GenericRecord> datumReader = new
GenericDatumReader<>(writerSchema, readerSchema);
+ final DataFileReader<GenericRecord> dataFileReader = new
DataFileReader<>(file.file(), datumReader);
Review comment:
`dataFileReader` should be registered on `closer`.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]