You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2019/05/09 23:41:21 UTC

[incubator-pinot] 01/01: Add SimpleAvroMessageDecoder which allows passing in Avro schema directly

This is an automated email from the ASF dual-hosted git repository.

jackie pushed a commit to branch simple-avro-message-decoder
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 578fbda4da6c60c36c5ee774e3062473b31097b3
Author: Jackie (Xiaotian) Jiang <xa...@linkedin.com>
AuthorDate: Thu May 9 16:37:37 2019 -0700

    Add SimpleAvroMessageDecoder which allows passing in Avro schema directly
    
    For cases where users have avro encoded msg in kafka but not yet setup a schema service to fetch schema on the fly.
    Contributor: fx19880617
---
 .../impl/kafka/SimpleAvroMessageDecoder.java       | 77 ++++++++++++++++++++++
 1 file changed, 77 insertions(+)

diff --git a/pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/kafka/SimpleAvroMessageDecoder.java b/pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/kafka/SimpleAvroMessageDecoder.java
new file mode 100644
index 0000000..56511b7
--- /dev/null
+++ b/pinot-core/src/main/java/org/apache/pinot/core/realtime/impl/kafka/SimpleAvroMessageDecoder.java
@@ -0,0 +1,77 @@
+/**
+ * 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.pinot.core.realtime.impl.kafka;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.util.Map;
+import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.pinot.common.data.Schema;
+import org.apache.pinot.core.data.GenericRow;
+import org.apache.pinot.core.realtime.stream.StreamMessageDecoder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+@NotThreadSafe
+public class SimpleAvroMessageDecoder implements StreamMessageDecoder<byte[]> {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SimpleAvroMessageDecoder.class);
+
+  private static final String SCHEMA = "schema";
+
+  private org.apache.avro.Schema _avroSchema;
+  private DatumReader<GenericData.Record> _datumReader;
+  private AvroRecordToPinotRowGenerator _avroRecordConverter;
+  private BinaryDecoder _binaryDecoderToReuse;
+  private GenericData.Record _avroRecordToReuse;
+
+  @Override
+  public void init(Map<String, String> props, Schema indexingSchema, String topicName)
+      throws Exception {
+    Preconditions.checkState(props.containsKey(SCHEMA), "Avro schema must be provided");
+    _avroSchema = new org.apache.avro.Schema.Parser().parse(props.get(SCHEMA));
+    _datumReader = new GenericDatumReader<>(_avroSchema);
+    _avroRecordConverter = new AvroRecordToPinotRowGenerator(indexingSchema);
+  }
+
+  @Override
+  public GenericRow decode(byte[] payload, GenericRow destination) {
+    return decode(payload, 0, payload.length, destination);
+  }
+
+  @Override
+  public GenericRow decode(byte[] payload, int offset, int length, GenericRow destination) {
+    if (payload == null || payload.length == 0 || length == 0) {
+      return null;
+    }
+    _binaryDecoderToReuse = DecoderFactory.get().binaryDecoder(payload, offset, length, _binaryDecoderToReuse);
+    try {
+      _avroRecordToReuse = _datumReader.read(_avroRecordToReuse, _binaryDecoderToReuse);
+    } catch (IOException e) {
+      LOGGER.error("Caught exception while reading message using schema: {}", _avroSchema, e);
+      return null;
+    }
+    return _avroRecordConverter.transform(_avroRecordToReuse, destination);
+  }
+}


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