You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/03/30 01:34:42 UTC

[GitHub] [druid] clintropolis commented on a change in pull request #11040: add avro stream input format

clintropolis commented on a change in pull request #11040:
URL: https://github.com/apache/druid/pull/11040#discussion_r603707036



##########
File path: extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamInputFormat.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.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.util.Objects;
+
+public class AvroStreamInputFormat extends NestedInputFormat
+{
+  private final boolean binaryAsString;
+
+  private final AvroBytesDecoder avroBytesDecoder;
+
+  @JsonCreator
+  public AvroStreamInputFormat(
+      @JsonProperty("flattenSpec") @Nullable JSONPathSpec flattenSpec,
+      @JsonProperty("avroBytesDecoder") AvroBytesDecoder avroBytesDecoder,
+      @JsonProperty("binaryAsString") @Nullable Boolean binaryAsString
+  )
+  {
+    super(flattenSpec);
+    this.avroBytesDecoder = avroBytesDecoder;
+    this.binaryAsString = binaryAsString == null ? false : binaryAsString;
+  }
+
+  @Override
+  public boolean isSplittable()
+  {
+    return false;
+  }
+
+  @JsonProperty
+  public AvroBytesDecoder getAvroBytesDecoder()
+  {
+    return avroBytesDecoder;
+  }
+
+  @Override
+  public InputEntityReader createReader(InputRowSchema inputRowSchema, InputEntity source, File temporaryDirectory)
+  {
+    return new AvroStreamReader(
+        inputRowSchema,
+        source,
+        avroBytesDecoder,
+        getFlattenSpec(),
+        binaryAsString
+    );
+  }
+
+  @Override
+  public boolean equals(final Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    final AvroStreamInputFormat that = (AvroStreamInputFormat) o;
+    return Objects.equals(getFlattenSpec(), that.getFlattenSpec()) &&
+        Objects.equals(avroBytesDecoder, that.avroBytesDecoder);
+  }
+
+  @Override
+  public int hashCode()
+  {
+    return Objects.hash(getFlattenSpec(), avroBytesDecoder);
+  }

Review comment:
       equality/hashcode should probably consider `binaryAsString` for their computations

##########
File path: extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroStreamReader.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.google.common.collect.Iterators;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.commons.io.IOUtils;
+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.CloseableIterators;
+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.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class AvroStreamReader extends IntermediateRowParsingReader<GenericRecord>
+{
+  private final InputRowSchema inputRowSchema;
+  private final InputEntity source;
+  private final AvroBytesDecoder avroBytesDecoder;
+  private final ObjectFlattener<GenericRecord> recordFlattener;
+
+  AvroStreamReader(
+      InputRowSchema inputRowSchema,
+      InputEntity source,
+      AvroBytesDecoder avroBytesDecoder,
+      JSONPathSpec flattenSpec,
+      boolean binaryAsString
+  )
+  {
+    this.inputRowSchema = inputRowSchema;
+    this.source = source;
+    this.avroBytesDecoder = avroBytesDecoder;
+    this.recordFlattener = ObjectFlatteners.create(flattenSpec, new AvroFlattenerMaker(false, binaryAsString));
+  }
+
+  @Override
+  protected CloseableIterator<GenericRecord> intermediateRowIterator() throws IOException
+  {
+    return CloseableIterators.withEmptyBaggage(
+        Iterators.singletonIterator(avroBytesDecoder.parse(ByteBuffer.wrap(IOUtils.toByteArray(source.open())
+        ))));

Review comment:
       nit: strange formatting (occasionally style bot doesn't pick stuff up)
   
   ```suggestion
       return CloseableIterators.withEmptyBaggage(
           Iterators.singletonIterator(avroBytesDecoder.parse(ByteBuffer.wrap(IOUtils.toByteArray(source.open()))))
       );
   ```




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org