diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/pom.xml b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/pom.xml deleted file mode 100644 index 154762e2..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/pom.xml +++ /dev/null @@ -1,120 +0,0 @@ - - - - - - - gimel-dataapi - com.paypal.gimel - 2.0.0-SNAPSHOT - ../../pom.xml - - 4.0.0 - - gimel-kafka-0.10 - 2.0.0-SNAPSHOT - - - - com.paypal.gimel - gimel-common - ${gimel.version}-SNAPSHOT - - - com.databricks - spark-avro_${scala.binary.version} - 3.2.0 - ${packaging.scope} - - - org.scalatest - scalatest_${scala.binary.version} - ${scalatest.version} - test - - - - - src/main/scala - src/test/scala - - - org.apache.maven.plugins - maven-shade-plugin - 3.0.0 - - - - com.google.common - gimel-shaded.com.google.common - - - com.sun.jersey - gimel-shaded.com.sun.jersey - - - - org.apache.hadoop - gimel-shaded.org.apache.hadoop - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - gimel-shading - package - - shade - - - - - - org.scalatest - scalatest-maven-plugin - 1.0 - - ${project.build.directory}/surefire-reports - . - WDF TestSuite.txt - - - - test - - test - - - - - - - - diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/DataSet.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/DataSet.scala deleted file mode 100644 index 93c614e3..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/DataSet.scala +++ /dev/null @@ -1,202 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka - -import scala.language.implicitConversions -import scala.reflect.runtime.universe._ - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.streaming.kafka010.OffsetRange - -import com.paypal.gimel.datasetfactory.GimelDataSet -import com.paypal.gimel.kafka.conf.KafkaClientConfiguration -import com.paypal.gimel.kafka.reader.KafkaBatchConsumer -import com.paypal.gimel.kafka.utilities.ImplicitZKCheckPointers._ -import com.paypal.gimel.kafka.utilities.ZooKeeperHostAndNodes -import com.paypal.gimel.kafka.writer.KafkaBatchProducer -import com.paypal.gimel.logger.Logger - -/** - * Concrete Implementation for Kafka DataSet - * - * @param sparkSession : SparkSession - */ - -class DataSet(sparkSession: SparkSession) extends GimelDataSet(sparkSession: SparkSession) { - - // GET LOGGER - val logger = Logger() - logger.info(s"Initiated --> ${this.getClass.getName}") - - var readTillOffsetRange: Option[Array[OffsetRange]] = None - var alreadyCheckPointed = false - // FIXME What happens if two users call read and write at the same time? Data race over conf? - private var conf: KafkaClientConfiguration = _ - - /** - * Saves Currently Read Offsets to Zookeeper - */ - def saveCheckPoint(): Unit = { - if (alreadyCheckPointed) { - logger.warning("Already Check-Pointed, Consume Again to Checkpoint !") - } else { - val zkNode = conf.zkCheckPoints - val zkHost = conf.zkHostAndPort - val zk = ZooKeeperHostAndNodes(zkHost, zkNode) - (zk, readTillOffsetRange.get).saveZkCheckPoint - alreadyCheckPointed = true - logger.info(s"Check-Point --> ${readTillOffsetRange.get.mkString("|")} | Success @ -> ${zk} ") - } - } - - /** - * Completely Clear the CheckPointed Offsets, leading to Read from Earliest offsets from Kafka - */ - def clearCheckPoint(): Unit = { - val zkNode = conf.zkCheckPoints - val zkHost = conf.zkHostAndPort - val zk = ZooKeeperHostAndNodes(zkHost, zkNode) - zk.deleteZkCheckPoint() - } - - /** Read Implementation for Kafka DataSet - * - * @param dataset Name of the PCatalog Data Set - * @param datasetProps - * props is the way to set various additional parameters for read and write operations in DataSet class - * Example Usecase : to read kafka from-to a certain offset range : One can set something like below - - * val props = Map("fromOffset" -> 10, "toOffset" -> 20) - * val data = Dataset(sc).read("flights.topic", props) - * @return DataFrame - */ - override def read(dataset: String, datasetProps: Map[String, Any]): DataFrame = { - - if (datasetProps.isEmpty) { - throw new DataSetException("Props Map Cannot be emtpy for KafkaDataSet Read.") - } - conf = new KafkaClientConfiguration(datasetProps) - val (data, toOffset) = KafkaBatchConsumer.consumeFromKakfa(sparkSession, conf) - alreadyCheckPointed = false - readTillOffsetRange = Some(toOffset) - data - } - - /** Write Implementation for Kafka DataSet - * - * @param dataset Name of the PCatalog Data Set - * @param dataFrame The DataFrame to write to target - * @param datasetProps - * props is the way to set various additional parameters for read and write operations in DataSet class - * Example Usecase : to write kafka with a specific parallelism : One can set something like below - - * val props = Map("parallelsPerPartition" -> 10) - * Dataset(sc).write(clientDataFrame, props) - * @return DataFrame - */ - - override def write(dataset: String, dataFrame: DataFrame, datasetProps: Map[String, Any]): DataFrame = { - - if (datasetProps.isEmpty) { - throw new DataSetException("Props Map Cannot be emtpy for KafkaDataSet Write.") - } - conf = new KafkaClientConfiguration(datasetProps) - KafkaBatchProducer.produceToKafka(conf, dataFrame) - dataFrame - } - - // Add Additional Supported types to this list as and when we support other Types of RDD - // Example to start supporting RDD[String], add to List < typeOf[Seq[Map[String, String]]].toString) > - override val supportedTypesOfRDD: List[String] = List[String](typeOf[String].toString, typeOf[Array[Byte]].toString) - - /** - * Function writes a given dataframe to the actual Target System (Example Hive : DB.Table | HBASE namespace.Table) - * - * @param dataset Name of the PCatalog Data Set - * @param rdd The RDD[T] to write into Target - * Note the RDD has to be typeCast to supported types by the inheriting DataSet Operators - * instance#1 : ElasticSearchDataSet may support just RDD[Seq(Map[String, String])], so Elastic Search must implement supported Type checking - * instance#2 : Kafka, HDFS, HBASE - Until they support an RDD operation for Any Type T : They throw Unsupporter Operation Exception & Educate Users Clearly ! - * @param datasetProps - * props is the way to set various additional parameters for read and write operations in DataSet class - * Example Usecase : to write kafka with a specific parallelism : One can set something like below - - * val props = Map("parallelsPerPartition" -> 10) - * Dataset(sc).write(clientDataFrame, props) - * @return RDD[T] - */ - def write[T: TypeTag](dataset: String, rdd: RDD[T], datasetProps: Map[String, Any]): RDD[T] = { - - if (!supportedTypesOfRDD.contains(typeOf[T].toString)) { - throw new UnsupportedOperationException(s"""Invalid RDD Type. Supported Types : ${supportedTypesOfRDD.mkString(" | ")}""") - } else { - if (datasetProps.isEmpty) { - throw new DataSetException("Props Map Cannot be emtpy for KafkaDataSet Write.") - } - conf = new KafkaClientConfiguration(datasetProps) - val rdd1: RDD[String] = rdd.asInstanceOf[RDD[String]] - KafkaBatchProducer.produceToKafka(conf, rdd1) - } - rdd - } - - /** - * - * @param dataset Name of the UDC Data Set - * @param dataSetProps - * * @return Boolean - */ - override def create(dataset: String, dataSetProps: Map[String, Any]): Unit = { - throw new Exception(s"DataSet create for kafka currently not Supported") - } - - /** - * - * @param dataset Name of the UDC Data Set - * @param dataSetProps - * * @return Boolean - */ - override def drop(dataset: String, dataSetProps: Map[String, Any]): Unit = { - throw new Exception(s"DataSet drop for kafka currently not Supported") - } - - /** - * - * @param dataset Name of the UDC Data Set - * @param dataSetProps - * * @return Boolean - */ - override def truncate(dataset: String, dataSetProps: Map[String, Any]): Unit = { - throw new Exception(s"DataSet truncate for kafka currently not Supported") - } -} - -/** - * Custom Exception for KafkaDataSet initiation errors - * - * @param message Message to Throw - * @param cause A Throwable Cause - */ -private class DataSetException(message: String, cause: Throwable) - extends RuntimeException(message) { - if (cause != null) { - initCause(cause) - } - - def this(message: String) = this(message, null) -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/DataStream.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/DataStream.scala deleted file mode 100644 index c1e85f6c..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/DataStream.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka - -import scala.language.implicitConversions - -import org.apache.spark.streaming.StreamingContext - -import com.paypal.gimel.datastreamfactory.{GimelDataStream, StreamingResult} -import com.paypal.gimel.kafka.conf.KafkaClientConfiguration -import com.paypal.gimel.kafka.reader.KafkaStreamConsumer -import com.paypal.gimel.logger.Logger - -class DataStream(streamingContext: StreamingContext) extends GimelDataStream(streamingContext: StreamingContext) { - - // GET LOGGER - val logger = Logger() - logger.info(s"Initiated --> ${this.getClass.getName}") - - /** - * Provides DStream for a given configuration - * - * @param dataset Kafka Topic Name - * @param datasetProps Map of K->V kafka Properties - * @return Tuple2 Of - - * Dstream[GenericRecord , Its Equivalent JSON String] - * A Function That Takes (SQLContext, RDD[GenericRecord]) , and returns a DataFrame - */ - def read(dataset: String, datasetProps: Map[String, Any]): StreamingResult = { - - if (datasetProps.isEmpty) { - throw new DataStreamException("Props Map Cannot be empty for KafkaDataSet Read") - } - val conf = new KafkaClientConfiguration(datasetProps) - KafkaStreamConsumer.createDStream(streamingContext, conf) - } - -} - -/** - * Custom Exception for KafkaDataStream initiation errors - * - * @param message Message to Throw - * @param cause A Throwable Cause - */ -private class DataStreamException(message: String, cause: Throwable) - extends RuntimeException(message) { - if (cause != null) { - initCause(cause) - } - - def this(message: String) = this(message, null) -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/avro/AvroToSQLSchemaConverter.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/avro/AvroToSQLSchemaConverter.scala deleted file mode 100644 index cf397e20..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/avro/AvroToSQLSchemaConverter.scala +++ /dev/null @@ -1,231 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.avro - -import java.nio.ByteBuffer -import java.util - -import scala.collection.JavaConverters._ - -import org.apache.avro.Schema -import org.apache.avro.Schema.Type._ -import org.apache.avro.generic.{GenericData, GenericRecord} -import org.apache.avro.generic.GenericData.Fixed -import org.apache.spark.sql.Row -import org.apache.spark.sql.types._ - -/** - * This looic is borrowed from databricks spark-avro-2_10.jar to aid in the conversion of avro RDD to DataFrame. - * - * https://github.com/databricks/spark-avro/blob/master/src/main/scala/com/databricks/spark/avro/SchemaConverters.scala - * - * This object contains method that are used to convert sparkSQL schemas to avro schemas and vice versa. - * - * Note that original code has been enhanced. Please ensure notes are maintained for new additions to track deviations from original code. - * - * 2017-08-19 : Added support for Set(STRING, LONG) : This enabled Reading FTPI data - */ -object AvroToSQLSchemaConverter { - - case class SchemaType(dataType: DataType, nullable: Boolean) - - /** - * This function takes an avro schema and returns a sql schema. - */ - def toSqlType(avroSchema: Schema): SchemaType = { - avroSchema.getType match { - case INT => - SchemaType(IntegerType, nullable = false) - case STRING => - SchemaType(StringType, nullable = false) - case BOOLEAN => - SchemaType(BooleanType, nullable = false) - case BYTES => - SchemaType(BinaryType, nullable = false) - case DOUBLE => - SchemaType(DoubleType, nullable = false) - case FLOAT => - SchemaType(FloatType, nullable = false) - case LONG => - SchemaType(LongType, nullable = false) - case FIXED => - SchemaType(BinaryType, nullable = false) - case ENUM => - SchemaType(StringType, nullable = false) - - case RECORD => - val fields = avroSchema.getFields.asScala.map { f => - val schemaType = toSqlType(f.schema()) - StructField(f.name, schemaType.dataType, schemaType.nullable) - } - - SchemaType(StructType(fields), nullable = false) - - case ARRAY => - val schemaType = toSqlType(avroSchema.getElementType) - SchemaType( - ArrayType(schemaType.dataType, containsNull = schemaType.nullable), - nullable = false) - - case MAP => - val schemaType = toSqlType(avroSchema.getValueType) - SchemaType( - MapType(StringType, schemaType.dataType, valueContainsNull = schemaType.nullable), - nullable = false) - - case UNION => - if (avroSchema.getTypes.asScala.exists(_.getType == NULL)) { - // In case of a union with null, eliminate it and make a recursive call - val remainingUnionTypes = avroSchema.getTypes.asScala.filterNot(_.getType == NULL).toList - if (remainingUnionTypes.size == 1) { - toSqlType(remainingUnionTypes.head).copy(nullable = true) - } else { - toSqlType(Schema.createUnion(remainingUnionTypes.asJava)).copy(nullable = true) - } - } else avroSchema.getTypes.asScala.map(_.getType) match { - case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => - SchemaType(LongType, nullable = false) - case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => - SchemaType(DoubleType, nullable = false) - case other => - throw new UnsupportedOperationException( - s"This mix of union types is not supported (see README): $other") - } - - case other => - throw new UnsupportedOperationException(s"Unsupported type $other") - } - } - - /** - * Returns a function that is used to convert avro types to their - * corresponding sparkSQL representations. - */ - def createConverterToSQL(schema: Schema): Any => Any = { - schema.getType match { - // Avro strings are in Utf8, so we have to call toString on them - case STRING | ENUM => - (item: Any) => if (item == null) null else item.toString - case INT | BOOLEAN | DOUBLE | FLOAT | LONG => - identity - // Byte arrays are reused by avro, so we have to make a copy of them. - case FIXED => - (item: Any) => - if (item == null) { - null - } else { - item.asInstanceOf[Fixed].bytes().clone() - } - case BYTES => - (item: Any) => - if (item == null) { - null - } else { - val bytes = item.asInstanceOf[ByteBuffer] - val javaBytes = new Array[Byte](bytes.remaining) - bytes.get(javaBytes) - javaBytes - } - case RECORD => - val fieldConverters = schema.getFields.asScala.map(f => createConverterToSQL(f.schema)) - (item: Any) => - if (item == null) { - null - } else { - val record = item.asInstanceOf[GenericRecord] - val converted = new Array[Any](fieldConverters.size) - var idx = 0 - while (idx < fieldConverters.size) { - converted(idx) = fieldConverters.apply(idx)(record.get(idx)) - idx += 1 - } - Row.fromSeq(converted.toSeq) - } - case ARRAY => - val elementConverter = createConverterToSQL(schema.getElementType) - (item: Any) => - if (item == null) { - null - } else { - item.asInstanceOf[GenericData.Array[Any]].asScala.map(elementConverter) - } - case MAP => - val valueConverter = createConverterToSQL(schema.getValueType) - (item: Any) => - if (item == null) { - null - } else { - item.asInstanceOf[util.HashMap[Any, Any]].asScala.map { case (k, v) => - (k.toString, valueConverter(v)) - }.toMap - } - case UNION => - if (schema.getTypes.asScala.exists(_.getType == NULL)) { - val remainingUnionTypes = schema.getTypes.asScala.filterNot(_.getType == NULL) - if (remainingUnionTypes.size == 1) { - createConverterToSQL(remainingUnionTypes.head) - } else { - createConverterToSQL(Schema.createUnion(remainingUnionTypes.asJava)) - } - } else schema.getTypes.asScala.map(_.getType) match { - case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => - (item: Any) => { - item match { - case l: Long => - l - case i: Int => - i.toLong - case null => - null - } - } - case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => - (item: Any) => { - item match { - case d: Double => - d - case f: Float => - f.toDouble - case null => - null - } - } - case Seq(t1, t2) if Set(t1, t2) == Set(STRING, LONG) => - (item: Any) => { - // @todo This fix is pending as currently we are unable to convert Avro to Spark types for this combination (STRING, LONG). Wip ! - item match { - case l: Long => - l - case js: org.apache.avro.util.Utf8 => - js.toString - case null => - null - } - } - case other => - throw new UnsupportedOperationException( - s"This mix of union types is not supported (see README): $other") - } - case other => - throw new UnsupportedOperationException(s"invalid avro type: $other") - } - } - -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/avro/SparkAvroUtilities.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/avro/SparkAvroUtilities.scala deleted file mode 100644 index 7a50fbfb..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/avro/SparkAvroUtilities.scala +++ /dev/null @@ -1,326 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.avro - -import java.io.{ByteArrayInputStream, ByteArrayOutputStream} - -import io.confluent.kafka.schemaregistry.client.rest.RestService -import org.apache.avro.{specific, Schema} -import org.apache.avro.generic.{GenericData, GenericRecord} -import org.apache.avro.io.{DecoderFactory, EncoderFactory} -import org.apache.avro.specific.SpecificDatumWriter -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row, SQLContext} -import org.apache.spark.sql.types.StructType -import scala.collection.JavaConverters._ -import spray.json._ -import spray.json.DefaultJsonProtocol._ - -import com.paypal.gimel.kafka.conf.KafkaClientConfiguration -import com.paypal.gimel.logger.Logger - -/** - * Avro - Spark Conversion operations are implemented here - */ - -object SparkAvroUtilities { - - val logger = Logger() - - /** - * Converts a DataFrame into RDD[Avro Generic Record] - * - * @param dataFrame DataFrame - * @param avroSchemaString Avro Schema String - * @return RDD[GenericRecord] - */ - - def dataFrametoGenericRecord(dataFrame: DataFrame, avroSchemaString: String): RDD[GenericRecord] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - try { - if (!isDFFieldsEqualAvroFields(dataFrame, avroSchemaString)) { - throw new SparkAvroConversionException(s"Incompatible DataFrame Schema Vs Provided Avro Schema.") - } - dataFrame.rdd.map { row => - val avroSchema = (new Schema.Parser).parse(avroSchemaString) - val fields = avroSchema.getFields.asScala.map { x => x.name() }.toArray - val cols: Map[String, Any] = row.getValuesMap(fields) - val genericRecord: GenericRecord = new GenericData.Record(avroSchema) - cols.foreach(x => genericRecord.put(x._1, x._2)) - genericRecord - } - } catch { - case ex: Throwable => - ex.printStackTrace() - throw new SparkAvroConversionException("Failed while converting DataFrame to Generic Record") - } - } - - /** - * Converts an RDD[Avro GenericRecord] into a DataFrame - * - * @param sqlContext SQLContext - * @param genericRecRDD RDD[GenericRecord] - * @param schemaString The AVRO schema String - * @return DataFrame - */ - def genericRecordtoDF(sqlContext: SQLContext, genericRecRDD: RDD[GenericRecord], schemaString: String): DataFrame = { - - genericRecordToDFViaAvroSQLConvertor(sqlContext, genericRecRDD, schemaString) - } - - /** - * Converts an RDD[Avro GenericRecord] into a DataFrame - * - * @param sqlContext SQLContext - * @param genericRecRDD RDD[GenericRecord] - * @param schemaString The AVRO schema String - * @return DataFrame - */ - - def genericRecordToDFViaAvroSQLConvertor(sqlContext: SQLContext, genericRecRDD: RDD[GenericRecord], schemaString: String): DataFrame = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - import com.databricks.spark.avro.SchemaConverters._ - try { - val rowRDD: RDD[Row] = genericRecRDD.map { x => - val avroSchema: Schema = (new Schema.Parser).parse(schemaString) - val converter = AvroToSQLSchemaConverter.createConverterToSQL(avroSchema) - converter(x).asInstanceOf[Row] - } - val avroSchema: Schema = (new Schema.Parser).parse(schemaString) - val schemaType = toSqlType(avroSchema) - sqlContext.createDataFrame(rowRDD, schemaType.dataType.asInstanceOf[StructType]) - } catch { - case ex: Throwable => - ex.printStackTrace() - throw new SparkAvroConversionException("Failed while converting Generic Record to DataFrame") - } - } - - /** - * Compare Fields of Avro Schema with Fields of DataFrame - * Return true if both match false if there is any mismatch - * Also log/print the differences. - * - * @param dataFrame DataFrame - * @param avroSchemaString Avro Schema String - * @return Boolean - */ - def isDFFieldsEqualAvroFields(dataFrame: DataFrame, avroSchemaString: String): Boolean = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - try { - val dfFields = dataFrame.schema.fieldNames - val avroSchema = (new Schema.Parser).parse(avroSchemaString) - val avroFields = avroSchema.getFields.asScala.map { x => x.name() }.toArray - val inDFMissingInAvro = dfFields.diff(avroFields) - val inAvroMissingInDF = avroFields.diff(dfFields) - val isMatching = inDFMissingInAvro.isEmpty && inAvroMissingInDF.isEmpty - if (!isMatching) { - val warningMessage = - s""" - |Provided Avro Fields --> ${avroFields.mkString(",")} - |Determined DataFrame Fields --> ${dfFields.mkString(",")} - |Missing Fields in Avro --> ${inDFMissingInAvro.mkString(",")} - |Missing Fields in DataFrame --> ${inAvroMissingInDF.mkString(",")} - """.stripMargin - logger.warning(warningMessage) - } - isMatching - } catch { - case ex: Throwable => - ex.printStackTrace() - throw new SparkAvroConversionException(s"Failed While Comparing DF Fields match against Fields in Avro Schema String $avroSchemaString") - } - - } - - /** - * Gets the fields from a Avro Schema String - * - * @param avroSchema Avro Schema String - * @return Fields - */ - def getFieldsFromAvroSchemaString(avroSchema: String): Seq[String] = { - val schemaAsJsVal = avroSchema.parseJson // parse as JsValue - val schemaAsJsObject = schemaAsJsVal.asJsObject // Convert to JsObject - val schemaFields = schemaAsJsObject.getFields("fields").head.convertTo[Seq[JsValue]] - val existingFields = schemaFields.map { x => x.asJsObject.fields("name").toString().replace("\"", "") } - existingFields - } - - /** - * DeSerialize an Avro Generic Record - * - * @param serializedBytes A Serialized Byte Array (serialization should have been done through Avro Serialization) - * @param schemaString An Avro Schema String - * @return An Avro Generic Record - */ - - def bytesToGenericRecord(serializedBytes: Array[Byte], schemaString: String): GenericRecord = { - - try { - // Build Avro Schema From String - val avroSchema = (new Schema.Parser).parse(schemaString) - // Initiate AVRO Reader from Factory - val reader = new specific.SpecificDatumReader[GenericRecord](avroSchema) - // Initiate a new Java Byte Array Input Stream - val in = new ByteArrayInputStream(serializedBytes) - // Get appropriate AVRO Decoder from Factory - val decoder = DecoderFactory.get().binaryDecoder(in, null) - // Get AVRO generic record - val genericRecordRead = reader.read(null, decoder) - genericRecordRead - } catch { - case ex: Throwable => - ex.printStackTrace() - throw ex - } - } - - /** - * Copies to a new generic record - * - * @param genericRecord Input Generic Record - * @param avroSchemaString Avro Schema that can be used to parse input Generic Record - * @param newAvroString New Avro Schema for the Outgoing Generic Record - * @return Outgoing Generic Record copied from Input - */ - def copyToGenericRecord(genericRecord: GenericRecord, avroSchemaString: String, newAvroString: String): GenericRecord = { - val existingFields = getFieldsFromAvroSchemaString(avroSchemaString) - val newAvroSchema = (new Schema.Parser).parse(newAvroString) - val newGenericRec: GenericRecord = new GenericData.Record(newAvroSchema) - existingFields.foreach(field => newGenericRec.put(field, genericRecord.get(field))) - newGenericRec - } - - /** - * A Functionality to Perform 2nd level De Serialization in case the data is from CDH - * This is necessary since Actual Data in CDH is wrapped by a Raw Record which get Deserialized when read from Kafka - * When this functionality is called, we check if the data is CDH type, then do second level deserialization - * If the data is not of CDH type, then we skip 2nd level deserialization - * - * @param avroRecordRDD RDD[GenericRecord] - * @param conf KafkaClientConfiguration - * @return RDD[GenericRecord] - */ - def deserializeCurrentRecord(avroRecordRDD: RDD[GenericRecord], conf: KafkaClientConfiguration): RDD[GenericRecord] = { - val schemaRegistryClient = new RestService(conf.avroSchemaURL) - val schemaLookup: scala.collection.mutable.Map[Int, String] = scala.collection.mutable.Map() - val actualRecord = avroRecordRDD.map { eachRecord => - val eachRecordSchemaVersion: Int = eachRecord.get("schemaVersion").toString.toInt - val schemaForThisRecord = schemaLookup.get(eachRecordSchemaVersion) match { - case None => - val schema = schemaRegistryClient.getVersion(conf.avroSchemaKey, eachRecordSchemaVersion).getSchema - schemaLookup.put(eachRecordSchemaVersion, schema) - schema - case Some(x) => - x - } - - val eachRecordBytes: Array[Byte] = eachRecord.get("currentRecord").asInstanceOf[Array[Byte]] - bytesToGenericRecord(eachRecordBytes, schemaForThisRecord) - } - actualRecord - } - - /** - * Serialize Avro GenericRecord into Byte Array - * - * @param rec An Avro Generic Record - * @param schemaString An Avro Schema String - * @return Serialized Byte Array - */ - - def genericRecordToBytes(rec: GenericRecord, schemaString: String): Array[Byte] = { - - try { - // Build Avro Schema From String - val avroSchema = (new Schema.Parser).parse(schemaString) - // Initiate a new Java Byte Array Output Stream - val out = new ByteArrayOutputStream() - // Get appropriate AVRO Decoder from Factory - val encoder = EncoderFactory.get().binaryEncoder(out, null) - // Write the Encoded data's output (Byte Array) into the Output Stream - // Initiate AVRO Writer from Factory - val writer = new SpecificDatumWriter[GenericRecord](avroSchema) - writer.write(rec, encoder) - // Flushes Data to Actual Output Stream - encoder.flush() - // Close the Output Stream - out.close() - val serializedBytes: Array[Byte] = out.toByteArray - serializedBytes - } catch { - case ex: Throwable => - ex.printStackTrace() - throw ex - } - } - - /** - * Converts an RDD[Avro GenericRecord] into a DataFrame - * - * @param sqlContext SQLContext - * @param genericRecRDD RDD[GenericRecord] - * @param schemaString The AVRO schema String - * @return DataFrame - */ - def genericRecordToDataFrameViaJSON(sqlContext: SQLContext, genericRecRDD: RDD[GenericRecord], schemaString: String): DataFrame = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - try { - val avroSchema: Schema = (new Schema.Parser).parse(schemaString) - val fields: Seq[String] = avroSchema.getFields.asScala.map { x => x.name() }.toArray.toSeq - sqlContext.read.json(genericRecRDD.map(_.toString)).selectExpr(fields: _*) - } catch { - case ex: Throwable => - ex.printStackTrace() - throw new SparkAvroConversionException("Failed while converting Generic Record to DataFrame") - } - } - - /** - * Custom Exception - * - * @param message Message to Throw - * @param cause A Throwable Cause - */ - private class SparkAvroConversionException(message: String, cause: Throwable) - extends RuntimeException(message) { - if (cause != null) { - initCause(cause) - } - - def this(message: String) = this(message, null) - } - -} - - diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaClientConfiguration.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaClientConfiguration.scala deleted file mode 100644 index 41f472b7..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaClientConfiguration.scala +++ /dev/null @@ -1,188 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.conf - -import java.util.Properties - -import scala.collection.JavaConverters._ -import scala.collection.immutable.Map -import scala.language.implicitConversions - -import io.confluent.kafka.schemaregistry.client.rest.RestService - -import com.paypal.gimel.common.catalog.DataSetProperties -import com.paypal.gimel.common.conf.{CatalogProviderConstants, GimelConstants, GimelProperties} -import com.paypal.gimel.common.schema.SchemaRegistryLookUp -import com.paypal.gimel.logger.Logger - -/** - * Gimel Client Configuration for Kafka Dataset Operations. - * - * @param props Kafka Client properties. - */ -class KafkaClientConfiguration(val props: Map[String, Any]) { - - private val logger = Logger() - logger.info(s"Begin Building --> ${this.getClass.getName}") - // logger.info(s"Incoming Properties --> ${props.map(x => s"${x._1} -> ${x._2}").mkString("\n")}") - - // Load Default Prop from Resource File - val pcatProps = GimelProperties() - - // appTag is used to maintain checkpoints & various other factors that are unique to the application - val appTag: String = props.getOrElse(GimelConstants.APP_TAG, "").toString - - // This is the DataSet Properties - val datasetProps: DataSetProperties = props(GimelConstants.DATASET_PROPS).asInstanceOf[DataSetProperties] - val tableProps: Map[String, String] = datasetProps.props - val hiveDBName = tableProps.getOrElse(CatalogProviderConstants.PROPS_NAMESPACE, GimelConstants.PCATALOG_STRING) - val hiveTableName = tableProps(CatalogProviderConstants.DATASET_PROPS_DATASET) - val clusterName = props.getOrElse(KafkaConstants.cluster, "unknown") - - logger.info(s"Hive Table Props --> ${tableProps.map(x => s"${x._1} --> ${x._2}").mkString("\n")}") - - // Schema Source either comes from Table "INLINE" (as a property) or from confluent Schema Registry if its = "CDH" or "CSR" - val avroSchemaSource: String = tableProps.getOrElse(KafkaConfigs.avroSchemaSource, KafkaConstants.gimelKafkaAvroSchemaInline) - val avroSchemaURL: String = tableProps.getOrElse(KafkaConfigs.avroSchemaSourceUrl, pcatProps.confluentSchemaURL) - val avroSchemaWrapperKey: String = tableProps.getOrElse(KafkaConfigs.avroSchemaSourceWrapperKey, pcatProps.kafkaAvroSchemaKey) - val avroSchemaKey: String = tableProps.getOrElse(KafkaConfigs.avroSchemaSourceKey, "") - val (avroSchemaString, cdhTopicSchemaMetadata, cdhAllSchemaDetails) = - avroSchemaSource.toUpperCase() match { - case KafkaConstants.gimelKafkaAvroSchemaCDH => - val schemaRegistryClient = new RestService(avroSchemaURL) - val allSchemas = SchemaRegistryLookUp.getAllSubjectAndSchema(avroSchemaURL) - (schemaRegistryClient.getLatestVersion(avroSchemaWrapperKey).getSchema, - Some(allSchemas(avroSchemaKey)._1), - Some(allSchemas) - ) - case KafkaConstants.gimeKafkaAvroSchemaCSR => - val schemaRegistryClient = new RestService(avroSchemaURL) - (schemaRegistryClient.getLatestVersion(avroSchemaWrapperKey).getSchema, - None, - None - ) - case KafkaConstants.gimelKafkaAvroSchemaInline => - (tableProps.getOrElse(KafkaConfigs.avroSchemaStringKey, ""), None, None) - case _ => - throw new Exception(s"Unsupported Schema Source Supplied --> $avroSchemaSource") - } - - // Kafka Props - val randomId: String = scala.util.Random.nextInt.toString - val kafkaHostsAndPort: String = tableProps.getOrElse(KafkaConfigs.kafkaServerKey, pcatProps.kafkaBroker) - val KafkaConsumerGroupID: String = props.getOrElse(KafkaConfigs.kafkaGroupIdKey, tableProps.getOrElse(KafkaConfigs.kafkaGroupIdKey, randomId)).toString - val kafkaConsumerID: String = props.getOrElse(KafkaConfigs.consumerId, tableProps.getOrElse(KafkaConfigs.consumerId, appTag)).toString.replaceAllLiterally("/", "_").replaceAllLiterally(":", "_") - val kafkaZKTimeOutMilliSec: String = tableProps.getOrElse(KafkaConfigs.zookeeperConnectionTimeoutKey, 10000.toString) - val kafkaAutoOffsetReset: String = tableProps.getOrElse(KafkaConfigs.offsetResetKey, "smallest") - val kafkaCustomOffsetRange: String = tableProps.getOrElse(KafkaConfigs.customOffsetRange, "") - val consumerModeBatch: String = tableProps.getOrElse(KafkaConstants.gimelAuditRunTypeBatch, "BATCH") - val consumerModeStream: String = tableProps.getOrElse(KafkaConstants.gimelAuditRunTypeStream, "STREAM") - val kafkaTopics: String = tableProps.getOrElse(KafkaConfigs.whiteListTopicsKey, "") - - // Kafka Serde - val kafkaKeySerializer: String = tableProps.getOrElse(KafkaConfigs.serializerKey, KafkaConfigs.kafkaStringSerializer) - val kafkaValueSerializer: String = tableProps.getOrElse(KafkaConfigs.serializerValue, KafkaConfigs.kafkaByteSerializer) - val kafkaKeyDeSerializer: String = tableProps.getOrElse(KafkaConfigs.deSerializerKey, KafkaConfigs.kafkaStringDeSerializer) - val kafkaValueDeSerializer: String = tableProps.getOrElse(KafkaConfigs.deSerializerValue, KafkaConfigs.kafkaByteDeSerializer) - - // Kafka Message Value Type --> String, Byte, Avro, JSON - val kafkaMessageValueType: Option[String] = tableProps.get(KafkaConfigs.kafkaMessageValueType) - - // Zookeeper Details - val zkHostAndPort: String = tableProps.getOrElse(KafkaConfigs.zookeeperCheckpointHost, pcatProps.zkHostAndPort) - if (pcatProps.kafkaConsumerCheckPointRoot == "") throw new Exception("Root CheckPoint Path for ZK cannot be Empty") - if (appTag == "") throw new Exception("appTag cannot be Empty") - if (kafkaTopics == "") throw new Exception("kafkaTopics cannot be Empty") - val zkCheckPoints: Seq[String] = kafkaTopics.split(",").map{ kafkaTopic => - tableProps.getOrElse(KafkaConfigs.zookeeperCheckpointPath, pcatProps.kafkaConsumerCheckPointRoot) + "/" + appTag + "/" + kafkaTopic - } - - // Kafka Monitoring for PayPal - /* - val kafkaMetricsReporter = props.getOrElse(KafkaConfigs.paypalMetricsReporterKey, KafkaConfigs.paypalMetricsReporterValue).toString - val kafkaMonitoringCluster = props.getOrElse(KafkaConfigs.paypalKafkaClusterKey, "unknown").toString - val kafkaMonitoringColo = props.getOrElse(KafkaConfigs.paypalMonitoringColoKey, "unknown").toString - val kafkaMonitoringPoolDefault = kafkaConsumerID - val kafkaMonitoringPool = "Gimel-" + props.getOrElse(KafkaConfigs.paypalMonitoringPoolKey, kafkaMonitoringPoolDefault).toString - val kafkaInterceptorClasses = props.getOrElse(KafkaConfigs.paypalInterceptorClassesKey, KafkaConfigs.paypalInterceptorClassName).toString - val kafkaMetricsSamplingWindowMilliSec = props.getOrElse(KafkaConfigs.paypalMetricsSamplingMilliSecKey, "6000").toString -*/ - val clientProps = scala.collection.immutable.Map( - KafkaConfigs.kafkaServerKey -> kafkaHostsAndPort - , KafkaConfigs.kafkaGroupIdKey -> s"${KafkaConsumerGroupID}" - , KafkaConfigs.kafkaClientIdKey -> s"${scala.util.Random.nextInt.toString}_${kafkaConsumerID}".takeRight(128) - ) - -// val ppKafkaListnerProps = scala.collection.immutable.Map( -// KafkaConfigs.paypalMetricsReporterKey -> kafkaMetricsReporter -// , KafkaConfigs.paypalKafkaClusterKey -> kafkaMonitoringCluster -// , KafkaConfigs.paypalMonitoringColoKey -> kafkaMonitoringColo -// , KafkaConfigs.paypalMonitoringPoolKey -> kafkaMonitoringPool -// , KafkaConfigs.paypalInterceptorClassesKey -> kafkaInterceptorClasses -// , KafkaConfigs.paypalMetricsSamplingMilliSecKey -> kafkaMetricsSamplingWindowMilliSec -// ) - - // Explicitly Making a Map of Properties that are necessary to Connect to Kafka for Subscribes (Reads) - val kafkaConsumerProps: Map[String, String] = scala.collection.immutable.Map(KafkaConfigs.kafkaServerKey -> kafkaHostsAndPort - , KafkaConfigs.kafkaGroupIdKey -> KafkaConsumerGroupID - , KafkaConfigs.zookeeperConnectionTimeoutKey -> kafkaZKTimeOutMilliSec - , KafkaConfigs.offsetResetKey -> kafkaAutoOffsetReset - , KafkaConfigs.kafkaTopicKey -> kafkaTopics - , KafkaConfigs.serializerKey -> kafkaKeySerializer - , KafkaConfigs.serializerValue -> kafkaValueSerializer - , KafkaConfigs.deSerializerKey -> kafkaKeyDeSerializer - , KafkaConfigs.deSerializerValue -> kafkaValueDeSerializer - ) ++ clientProps - - logger.info(s"KafkaConsumerProps --> ${kafkaConsumerProps.mkString("\n")}") - - // Explicitly Making a Map of Properties that are necessary to Connect to Kafka for Publishes (Writes) - val kafkaProducerProps: Properties = new java.util.Properties() - val producerProps = scala.collection.immutable.Map(KafkaConfigs.kafkaServerKey -> kafkaHostsAndPort - , KafkaConfigs.serializerKey -> kafkaKeySerializer - , KafkaConfigs.serializerValue -> kafkaValueSerializer - , KafkaConfigs.kafkaTopicKey -> kafkaTopics) - producerProps.foreach { kvPair => kafkaProducerProps.put(kvPair._1.toString, kvPair._2.toString) } - - logger.info(s"kafkaProducerProps --> ${kafkaProducerProps.asScala.mkString("\n")}") - - // These are key throttling factors for Improved Performance in Batch Mode - val maxRecsPerPartition: Long = props.getOrElse(KafkaConfigs.maxRecordsPerPartition, 2500000).toString.toLong - val parallelsPerPartition: Int = props.getOrElse(KafkaConfigs.batchFetchSizeTemp, 250).toString.toInt - val minRowsPerParallel: Long = props.getOrElse(KafkaConfigs.minRowsPerParallelKey, 100000).toString.toLong - val fetchRowsOnFirstRun: Long = props.getOrElse(KafkaConfigs.rowCountOnFirstRunKey, 2500000).toString.toLong - val targetCoalesceFactor: Int = props.getOrElse(KafkaConfigs.targetCoalesceFactorKey, 1).toString.toInt - - // These are key throttling factors for Improved Performance in Streaming Mode - val maxRatePerPartition: String = props.getOrElse(KafkaConfigs.maxRatePerPartitionKey, 3600).toString - val streamParallelismFactor: Int = props.getOrElse(KafkaConfigs.streamParallelKey, 10).toString.toInt - val isStreamParallel: Boolean = props.getOrElse(KafkaConfigs.isStreamParallelKey, "true").toString.toBoolean - - // Resolve fields for empty kafka topic property - val fieldsBindToJSONString = tableProps.getOrElse(GimelConstants.FIELDS_BIND_TO_JSON, "") - - // Additional CDH Metadata Fields @todo this is not used in the code yet, KafkaUtilities implements this inside - this must superceed everywhere. - val additionalCDHFields = scala.collection.Map("gg_commit_timestamp" -> "opTs", "opt_type" -> "opType", "trail_seq_no" -> "trailSeqno", "trail_rba" -> "trailRba") - - logger.info(s"Fields Initiated --> ${this.getClass.getFields.map(f => s"${f.getName} --> ${f.get().toString}").mkString("\n")}") - logger.info(s"Completed Building --> ${this.getClass.getName}") - -} - diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaConfigs.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaConfigs.scala deleted file mode 100644 index c47cfc5b..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaConfigs.scala +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.conf - -object KafkaConfigs { - - // kafka properties - val kafkaServerKey: String = "bootstrap.servers" - val kafkaGroupIdKey: String = "group.id" - val kafkaClientIdKey: String = "client.id" - val paypalMetricsReporterKey: String = "metric.reporters" - val paypalKafkaClusterKey: String = "kafka.monitoring.cluster" - val paypalMonitoringColoKey: String = "kafka.monitoring.colo" - val paypalMonitoringPoolKey: String = "kafka.monitoring.pool" - val paypalInterceptorClassesKey: String = "interceptor.classes" - val paypalMetricsSamplingMilliSecKey: String = "metrics.sample.window.ms" - val zookeeperConnectionTimeoutKey: String = "zookeeper.connection.timeout.ms" - val offsetResetKey: String = "auto.offset.reset" - val kafkaTopicKey: String = "kafka.topic" - val serializerKey: String = "key.serializer" - val serializerValue: String = "value.serializer" - val deSerializerKey: String = "key.deserializer" - val deSerializerValue: String = "value.deserializer" - val consumerId: String = "consumer.id" - // misc properties for read/write - val rowCountOnFirstRunKey: String = s"gimel.kafka.throttle.batch.fetchRowsOnFirstRun" - val targetCoalesceFactorKey: String = "gimel.kafka.throttle.batch.targetCoalesceFactor" - val minRowsPerParallelKey: String = s"gimel.kafka.throttle.batch.minRowsPerParallel" - val batchFetchSize: String = s"gimel.kafka.throttle.batch.parallelsPerPartition" - val maxRecordsPerPartition: String = s"gimel.kafka.throttle.batch.maxRecordsPerPartition" - val batchFetchSizeTemp: String = s"gimel.kafka.throttle.batch.parallelsPerPartition" - val messageColumnAliasKey: String = "gimel.kafka.message.column.alias" - val avroSchemaStringKey: String = "gimel.kafka.avro.schema.string" - val kafkaMessageValueType: String = "gimel.kafka.message.value.type" - // metastore properties - val zookeeperCheckpointHost: String = "gimel.kafka.checkpoint.zookeeper.host" - val zookeeperCheckpointPath: String = "gimel.kafka.checkpoint.zookeeper.path" - val avroSchemaSource: String = "gimel.kafka.avro.schema.source" - val avroSchemaSourceUrl: String = s"${avroSchemaSource}.url" - val avroSchemaSourceWrapperKey: String = s"${avroSchemaSource}.wrapper.key" - val avroSchemaSourceKey: String = s"${avroSchemaSource}.key" - val whiteListTopicsKey: String = "gimel.kafka.whitelist.topics" - // streaming properties - val defaultBatchInterval: String = "gimel.kafka.throttle.streaming.window.seconds" - val maxRatePerPartitionKey: String = "gimel.kafka.throttle.streaming.maxRatePerPartition" - val streamMaxRatePerPartitionKey: String = "gimel.kafka.spark.streaming.kafka.maxRatePerPartition" - val streamParallelKey: String = "gimel.kafka.throttle.streaming.parallelism.factor" - val isStreamParallelKey: String = "gimel.kafka.throttle.streaming.isParallel" - val isBackPressureEnabledKey: String = "gimel.kafka.spark.streaming.backpressure.enabled" - val streamaWaitTerminationOrTimeoutKey: String = "gimel.kafka.streaming.awaitTerminationOrTimeout" - val isStreamBatchSwitchEnabledKey: String = "gimel.kafka.stream.batch.switch.enabled" - val failStreamThresholdKey: String = "gimel.kafka.fail.stream.threshold.message.per.second" - val streamCutOffThresholdKey: String = "gimel.kafka.batch.to.stream.cutoff.threshold" - val streamFailureThresholdPerSecondKey: String = "gimel.kafka.fail.stream.threshold.message.per.second" - val streamFailureWindowFactorKey: String = "gimel.kafka.fail.stream.window.factor" - val kafkaConsumerReadCheckpointKey: String = "gimel.kafka.reader.checkpoint.save" - val kafkaConsumerClearCheckpointKey: String = "gimel.kafka.reader.checkpoint.clear" - val customOffsetRange: String = "gimel.kafka.custom.offset.range" - // default packages used in Kafka read/write API - val paypalMetricsReporterValue: String = "com.paypal.kafka.reporters.KafkaClientMetricsReporter" - val paypalInterceptorClassName: String = "com.paypal.kafka.clients.interceptors.MonitoringConsumerInterceptor" - val kafkaStorageHandler: String = "org.apache.hadoop.hive.kafka.KafkaStorageHandler" - val kafkaStringSerializer: String = "org.apache.kafka.common.serialization.StringSerializer" - val kafkaByteSerializer: String = "org.apache.kafka.common.serialization.ByteArraySerializer" - val kafkaStringDeSerializer: String = "org.apache.kafka.common.serialization.StringDeserializer" - val kafkaByteDeSerializer: String = "org.apache.kafka.common.serialization.ByteArrayDeserializer" -} - diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaConstants.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaConstants.scala deleted file mode 100644 index bbd1f8f3..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaConstants.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.conf - -object KafkaConstants { - // basic variable references - val gimelKafkaAvroSchemaCDH = "CDH" - val gimeKafkaAvroSchemaCSR = "CSR" - val gimelKafkaAvroSchemaInline = "INLINE" - val gimelAuditRunTypeBatch = "BATCH" - val gimelAuditRunTypeStream = "STREAM" - val gimelAuditRunTypeIntelligent = "INTELLIGENT" - val cluster = "cluster" - // polling properties - val unknownContainerName = "unknown" - val kafkaAllTopics = "All" - val targetDb = "pcatalog" - val generateDdlKey = "generate_ddl_for" - val targetDbkey = "target_db" - val avroToHiveTypes = Map( - "null" -> "void", - "boolean" -> "boolean", - "int" -> "int", - "long" -> "bigint", - "float" -> "float", - "double" -> "double", - "bytes" -> "binary", - "string" -> "string", - "record" -> "struct", - "map" -> "map", - "list" -> "array", - "union" -> "union", - "enum" -> "string", - "fixed" -> "binary") - // STRUCTURED STREAMING SPARK CONSTANTS - val KAFKA_FORMAT: String = "org.apache.spark.sql.kafka010.KafkaSourceProvider" - val KAFKA_BOOTSTRAP_SERVERS: String = "kafka.bootstrap.servers" - val KAFKA_SUBSCRIBE: String = "subscribe" - val KAFKA_START_OFFSETS: String = "startingOffsets" - val KAFKA_END_OFFSETS: String = "endingOffsets" - val STREAM_FAIL_ON_DATA_LOSS: String = "failOnDataLoss" - val KAFKA_POLL_TIMEOUT: String = "kafkaConsumer.pollTimeoutMs" - val KAFKA_FETCH_RETRIES: String = "fetchOffset.numRetries" - val KAFKA_RETRY_INTERVAL: String = "fetchOffset.retryIntervalMs" - val earliestOffset: String = "earliest" -} - diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaJsonProtocol.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaJsonProtocol.scala deleted file mode 100644 index 200de228..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/conf/KafkaJsonProtocol.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.conf - -import spray.json.{DefaultJsonProtocol, RootJsonFormat} - -import com.paypal.gimel.kafka.utilities.{OffsetProperties, OffsetRangeProperties} - - -object KafkaJsonProtocol extends DefaultJsonProtocol { - implicit val offsetRangePropertiesFormat: RootJsonFormat[OffsetRangeProperties] = jsonFormat3(OffsetRangeProperties) - implicit val offsetPropertiesFormat: RootJsonFormat[OffsetProperties] = jsonFormat2(OffsetProperties) -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/reader/KafkaBatchConsumer.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/reader/KafkaBatchConsumer.scala deleted file mode 100644 index 8144311e..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/reader/KafkaBatchConsumer.scala +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.reader - -import scala.collection.immutable.Map -import scala.language.implicitConversions - -import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.streaming.kafka010.OffsetRange - -import com.paypal.gimel.common.conf.GimelConstants -import com.paypal.gimel.common.utilities.BindToFieldsUtils._ -import com.paypal.gimel.kafka.conf.{KafkaClientConfiguration, KafkaConstants} -import com.paypal.gimel.kafka.utilities.{BrokersAndTopic, KafkaUtilitiesException} -import com.paypal.gimel.kafka.utilities.ImplicitKafkaConverters._ -import com.paypal.gimel.kafka.utilities.KafkaUtilities._ - -/** - * Implements Kafka Consumer Batch Here - */ -object KafkaBatchConsumer { - - val logger = com.paypal.gimel.logger.Logger() - - - /** - * Connects to Kafka, Deserializes data from Kafka, Attempts to Convert Avro to a DataFrame - * - * @param sparkSession : SparkSession - * @param conf KafkaClientConfiguration - * @return DataFrame - * @return Read Till Array[OffsetRange] - * - */ - - def consumeFromKakfa(sparkSession: SparkSession, conf: KafkaClientConfiguration): (DataFrame, Array[OffsetRange]) = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - val kafkaParams: Map[String, String] = conf.kafkaConsumerProps - try { - val finalOffsetRangesForReader: Array[OffsetRange] = - if (conf.kafkaCustomOffsetRange.isEmpty()) { - logger.info(s"""No custom offset information was given by the user""") - val lastCheckPoint: Option[Array[OffsetRange]] = getLastCheckPointFromZK(conf.zkHostAndPort, conf.zkCheckPoints) - val availableOffsetRange: Array[OffsetRange] = BrokersAndTopic(conf.kafkaHostsAndPort, conf.kafkaTopics).toKafkaOffsetsPerPartition - val newOffsetRangesForReader = getNewOffsetRangeForReader(lastCheckPoint, availableOffsetRange, conf.fetchRowsOnFirstRun) - logger.info("Offset Ranges From Difference -->") - newOffsetRangesForReader.foreach(x => logger.info(x.toString)) - newOffsetRangesForReader.applyThresholdPerPartition(conf.maxRecsPerPartition.toLong) // Restrict Offset Ranges By Applying Threshold Per Partition - } - else { - logger.info(s"""Custom offset information was given by the user""") - getCustomOffsetRangeForReader(conf.kafkaTopics.split(","), conf.kafkaCustomOffsetRange, KafkaConstants.gimelAuditRunTypeBatch) - } - logger.info("Offset Ranges After applying Threshold Per Partition/Custom Offsets -->") - finalOffsetRangesForReader.foreach(x => logger.info(x.toString)) - - // If kafka topic is empty return empty dataframe with the columns in gimel.fields.bind.to.json prop - val finalDF = if (isKafkaTopicEmpty(finalOffsetRangesForReader) && !conf.fieldsBindToJSONString.isEmpty) { - logger.info("Kafka Topic is Empty.") - logger.info("Returning Datafame with fields in " + GimelConstants.FIELDS_BIND_TO_JSON) - getEmptyDFBindToFields(sparkSession, conf.fieldsBindToJSONString) - } else { - val parallelizedRanges: Array[OffsetRange] = finalOffsetRangesForReader.parallelizeOffsetRanges(conf.parallelsPerPartition, conf.minRowsPerParallel) - logger.info("Final Array of OffsetRanges to Fetch from Kafka --> ") - parallelizedRanges.foreach(range => logger.info(range)) - if (parallelizedRanges.isEmpty) throw new KafkaUtilitiesException("There is an issue ! No Offset Range From Kafka ... Is the topic having any message at all ?") - val sqlContext = sparkSession.sqlContext - getAsDFFromKafka(sqlContext, conf, parallelizedRanges) - } - - (finalDF, finalOffsetRangesForReader) - } catch { - case ex: Throwable => - ex.printStackTrace() - val messageString = - s""" - |kafkaParams --> ${kafkaParams.mkString(" \n ")} - """.stripMargin - logger.error(s"An Error While Attempting to Consume From Kafka with Parameters --> $messageString") - throw ex - } - } - - /** - * Checks if the given kafka topics are empty - * - * @param offsetRanges : array of OffsetRanges for the topics to check - * @return - * - */ - def isKafkaTopicEmpty(offsetRanges: Array[OffsetRange]): Boolean = { - offsetRanges.isEmpty || offsetRanges.forall (each => (each.untilOffset - each.fromOffset) == 0) - } -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/reader/KafkaStreamConsumer.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/reader/KafkaStreamConsumer.scala deleted file mode 100644 index e213cedd..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/reader/KafkaStreamConsumer.scala +++ /dev/null @@ -1,273 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.reader - -import scala.collection.immutable.Map -import scala.language.implicitConversions - -import org.apache.avro.generic.GenericRecord -import org.apache.kafka.clients.consumer._ -import org.apache.kafka.common.TopicPartition -import org.apache.spark.rdd.RDD -import org.apache.spark.sql._ -import org.apache.spark.sql.streaming.DataStreamReader -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.kafka010._ -import spray.json._ - -import com.paypal.gimel.common.catalog.GimelCatalogJsonProtocol._ -import com.paypal.gimel.datastreamfactory.{CheckPointHolder, StreamingResult, StructuredStreamingResult, WrappedData} -import com.paypal.gimel.kafka.avro.SparkAvroUtilities -import com.paypal.gimel.kafka.conf.{KafkaClientConfiguration, KafkaConstants} -import com.paypal.gimel.kafka.utilities.BrokersAndTopic -import com.paypal.gimel.kafka.utilities.ImplicitKafkaConverters._ -import com.paypal.gimel.kafka.utilities.KafkaUtilities._ - -/** - * Implements Kafka Stream Consumer Logic here - */ -object KafkaStreamConsumer { - - val logger = com.paypal.gimel.logger.Logger() - - /** - * - * Core Function to Provide Data Stream - * - * @param streamingContext StreamingContext - * @param conf KafkaClientConfiguration - * @return StreamingResult - */ - def createDStream(streamingContext: StreamingContext, conf: KafkaClientConfiguration): StreamingResult = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - try { - val sparkConf = streamingContext.sparkContext.getConf - val streamRate = sparkConf.get("throttle.streaming.maxRatePerPartition", conf.maxRatePerPartition) - streamingContext.sparkContext.getConf - .set("spark.streaming.backpressure.enabled", "true") - .set("spark.streaming.kafka.maxRatePerPartition", streamRate) - val isStreamParallel = sparkConf.get("throttle.streaming.isParallel", conf.isStreamParallel.toString).toBoolean - val streamParallels = sparkConf.get("throttle.streaming.parallelism.factor", conf.streamParallelismFactor.toString).toInt - logger.debug( - s""" - |isStreamParallel --> ${isStreamParallel} - |streamParallels --> ${streamParallels} - """.stripMargin) - // Resolve all the Properties & Determine Kafka CheckPoint before reading from Kafka - val (schemaString, kafkaTopic, brokers) = (conf.avroSchemaString, conf.kafkaTopics, conf.kafkaHostsAndPort) - logger.info(s"Zookeeper Server : ${conf.zkHostAndPort}") - logger.info(s"Zookeeper Checkpoint : ${conf.zkCheckPoints}") - val startOffsetsForStream: Map[TopicPartition, Long] = - getStartOffsets(conf, kafkaTopic, brokers) - var kafkaParams: Map[String, Object] = setKafkaParams(conf) - val consumerStrategy = ConsumerStrategies.Subscribe[Any, Any](kafkaTopic.split(",").toSet, kafkaParams, startOffsetsForStream) - val locationStrategy = LocationStrategies.PreferConsistent - logger.info( - s""" - |consumerStrategy --> ${consumerStrategy} - |locationStrategy --> ${locationStrategy.toString} - |Initiating createDirectStream with above Parameters... - """.stripMargin) - val msg: InputDStream[ConsumerRecord[Any, Any]] = KafkaUtils.createDirectStream(streamingContext, locationStrategy, consumerStrategy) - var offsetRanges = Array[OffsetRange]() - val messages1: DStream[WrappedData] = msg.transform { rdd => - offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - // CheckPointHolder().currentCheckPoint = offsetRanges - CheckPointHolder().setCurentCheckPoint(offsetRanges) - rdd - }.map { x => WrappedData(x.key(), x.value()) } - // CheckPointer Function - CheckPoints each window - val saveCheckPoint: (Array[OffsetRange]) => Boolean = inStreamCheckPoint(conf.zkHostAndPort, conf.zkCheckPoints, _) - // Convertor Function : takes Raw Data and Returns AvroGeneric Data - val bytesToGenericRDD: (RDD[WrappedData]) => RDD[GenericRecord] = - wrappedDataToAvro(_, conf.avroSchemaKey, conf.avroSchemaURL, conf.avroSchemaSource, conf.avroSchemaString, isStreamParallel, streamParallels, conf.cdhAllSchemaDetails) - val finalSchema = conf.avroSchemaSource.toUpperCase() match { - case "CDH" => addAdditionalFieldsToSchema(getAdditionalFields().keySet.toList, conf.cdhTopicSchemaMetadata.get) - case _ => conf.avroSchemaString - } - // Convertor Function - RDD[GenericRecord] => DataFrame - val genericRecToDF: (SQLContext, RDD[GenericRecord]) => DataFrame = SparkAvroUtilities.genericRecordtoDF(_, _, finalSchema) - // Provide Option to Clear CheckPoint - val deleteCheckPoint: (String) => Unit = clearCheckPoint(conf.zkHostAndPort, conf.zkCheckPoints, _: String) - // Provide Option to Get DataFrame for a Simple String Message from Kafka Topic - val columnAlias = kafkaMessageColumnAlias(conf) - // val wrappedDataToDF: (SQLContext, RDD[WrappedData]) => DataFrame = wrappedStringDataToDF(columnAlias, _, _) - val wrappedDatatoDF1: (SQLContext, RDD[WrappedData]) => DataFrame = rddToDF(_, conf.kafkaMessageValueType, conf.kafkaKeySerializer, conf.kafkaValueSerializer, _, "value", conf.avroSchemaString, conf.avroSchemaSource, conf.cdhTopicSchemaMetadata, conf.cdhAllSchemaDetails) - // Return a Wrapper of various functionalities to Client of this function - StreamingResult(messages1, bytesToGenericRDD, genericRecToDF, wrappedDatatoDF1, saveCheckPoint, deleteCheckPoint) - } - catch { - case ex: Throwable => { - ex.printStackTrace() - streamingContext.stop() - throw ex - } - } - } - - /** - * - * Function to set kafka parameters for stream - * - * @param conf KafkaClientConfiguration object that holds the configuration paremeters - * @return Kafka Parameters in a Map[String, Object] - */ - private def setKafkaParams(conf: KafkaClientConfiguration) = { - var kafkaParams: Map[String, Object] = Map() - conf.kafkaConsumerProps.foreach(x => kafkaParams += (x._1 -> x._2)) - val (keyDeSer, valDeSer) = (getSerDe(conf.kafkaKeyDeSerializer), getSerDe(conf.kafkaValueDeSerializer)) - kafkaParams += ("key.deserializer" -> keyDeSer, "value.deserializer" -> valDeSer) - kafkaParams - } - - /** - * - * Function to get the starting offsets for the stream to read from - * - * @param conf KafkaClientConfiguration object that holds the configuration paremeters - * @param kafkaTopic The kafkaTopics list to subscribe to - * @return Starting Offsets in a Map[TopicPartition, Long] - */ - private def getStartOffsets(conf: KafkaClientConfiguration, kafkaTopic: String, brokers: String) = { - if (conf.kafkaCustomOffsetRange.isEmpty()) { - val lastCheckPoint: Option[Array[OffsetRange]] = getLastCheckPointFromZK(conf.zkHostAndPort, conf.zkCheckPoints) - val availableOffsetRange: Array[OffsetRange] = BrokersAndTopic(brokers, kafkaTopic).toKafkaOffsetsPerPartition - if (lastCheckPoint == None) { - logger.info("No CheckPoint Found !") - if(conf.kafkaAutoOffsetReset.equals(KafkaConstants.earliestOffset)) { - logger.info("Fetching from the beginning") - availableOffsetRange.map { - x => (new TopicPartition(x.topic, x.partition) -> x.fromOffset) - }.toMap - } - else { - logger.info("Fetching from the latest offset") - availableOffsetRange.map { - x => (new TopicPartition(x.topic, x.partition) -> x.untilOffset) - }.toMap - } - } else { - logger.info(s"Found Checkpoint Value --> ${lastCheckPoint.get.mkString("|")}") - lastCheckPoint.get.map { - x => (new TopicPartition(x.topic, x.partition) -> x.untilOffset) - }.toMap - } - } - else { - val customOffsetRangesForStream: Array[OffsetRange] = getCustomOffsetRangeForReader(conf.kafkaTopics.split(","), conf.kafkaCustomOffsetRange, KafkaConstants.gimelAuditRunTypeStream) - customOffsetRangesForStream.map { - x => (new TopicPartition(x.topic, x.partition) -> x.fromOffset) - }.toMap - } - } - - /** - * - * Function to return the last saved checkpoint from zookeeper - * - * @param conf KafkaClientConfiguration object that holds the configuration paremeters - * @return Optional checkpoint Offsets in a Array[OffsetRange] - */ - private def getLastCheckPoint(conf: KafkaClientConfiguration) = { - val lastCheckPoint: Option[Array[OffsetRange]] = getLastCheckPointFromZK(conf.zkHostAndPort, conf.zkCheckPoints) - lastCheckPoint - } - - /** - * - * Core Function to create a structured stream - * - * @param sparkSession the spark session passed by the user - * @param conf KafkaClientConfiguration object that holds the configuration paremeters - * @return StreamingResult in a StructuredStreamingResult Object - */ - def createStructuredStream(sparkSession: SparkSession, conf: KafkaClientConfiguration): StructuredStreamingResult = { - try { - val sparkConf = sparkSession.sparkContext.getConf - val streamRate = sparkConf.get("throttle.streaming.maxRatePerPartition", conf.maxRatePerPartition) - sparkSession.sparkContext.getConf - .set("spark.streaming.backpressure.enabled", "true") - .set("spark.streaming.kafka.maxRatePerPartition", streamRate) - val isStreamParallel = sparkConf.get("throttle.streaming.isParallel", conf.isStreamParallel.toString).toBoolean - val streamParallels = sparkConf.get("throttle.streaming.parallelism.factor", conf.streamParallelismFactor.toString).toInt - logger.debug( - s""" - |isStreamParallel --> ${isStreamParallel} - |streamParallels --> ${streamParallels} - """.stripMargin) - // Resolve all the Properties & Determine Kafka CheckPoint before reading from Kafka - val (schemaString, kafkaTopic, brokers) = (conf.avroSchemaString, conf.kafkaTopics, conf.kafkaHostsAndPort) - logger.info(s"Zookeeper Server : ${conf.zkHostAndPort}") - logger.info(s"Zookeeper Checkpoint : ${conf.zkCheckPoints}") - val startOffsetsForStream: Map[TopicPartition, Long] = - getStartOffsets(conf, kafkaTopic, brokers) - val lastCheckPoint = getLastCheckPoint(conf) - val startOffsetsStructured = startOffsetsForStream.toList.groupBy(_._1.topic()) - .mapValues(_.map(x => - (x._1.partition().toString, x._2)).toMap) - val kafkaBootstrapServers = conf.kafkaHostsAndPort - val topics = conf.kafkaTopics - - val dataStreamReader: DataStreamReader = sparkSession - .readStream - .format(KafkaConstants.KAFKA_FORMAT) - .option(KafkaConstants.KAFKA_BOOTSTRAP_SERVERS, kafkaBootstrapServers) - .option(KafkaConstants.KAFKA_SUBSCRIBE, topics) - .options(conf.kafkaConsumerProps) - - val df = lastCheckPoint match { - case None => { - dataStreamReader.load() - } - case Some(lastCheckPoint) => { - dataStreamReader - .option(KafkaConstants.KAFKA_START_OFFSETS, startOffsetsStructured.toJson.toString()) - .load() - } - } - - // CheckPointer Function - CheckPoints each window - val saveCheckPoint: Unit = inStructuredStreamCheckPoint(sparkSession, conf.zkHostAndPort, conf.zkCheckPoints) - // Convertor Function : takes Raw Data and Returns AvroGeneric Data - val bytesToGenericRDD: (RDD[WrappedData]) => RDD[GenericRecord] = - wrappedDataToAvro(_, conf.avroSchemaKey, conf.avroSchemaURL, conf.avroSchemaSource, conf.avroSchemaString, isStreamParallel, streamParallels, conf.cdhAllSchemaDetails) - val finalSchema = conf.avroSchemaSource.toUpperCase() match { - case "CDH" => addAdditionalFieldsToSchema(getAdditionalFields().keySet.toList, conf.cdhTopicSchemaMetadata.get) - case _ => conf.avroSchemaString - } - // Provide Option to Clear CheckPoint - val deleteCheckPoint: (String) => Unit = clearCheckPoint(conf.zkHostAndPort, conf.zkCheckPoints, _: String) - // Provide Option to Get DataFrame for a Simple String Message from Kafka Topic - val columnAlias = kafkaMessageColumnAlias(conf) - // Return a Wrapper of various functionalities to Client of this function - StructuredStreamingResult(df, saveCheckPoint, deleteCheckPoint) - } - - catch { - case ex: Throwable => { - ex.printStackTrace() - throw ex - } - } - } -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/ImplicitHDFSCheckPointers.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/ImplicitHDFSCheckPointers.scala deleted file mode 100644 index 30bbe0e4..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/ImplicitHDFSCheckPointers.scala +++ /dev/null @@ -1,127 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.utilities - -import scala.language.implicitConversions - -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.streaming.kafka010.OffsetRange - -import com.paypal.gimel.common.storageadmin.HDFSAdminClient._ -import com.paypal.gimel.kafka.utilities.ImplicitKafkaConverters._ - -/** - * Provides Implicit, Convenience Functions for Developers to Do CheckPointing Operations - */ -object ImplicitHDFSCheckPointers { - - val logger = com.paypal.gimel.logger.Logger() - - /** - * @param offsetRangesAndCheckPointDirectory A Tuple of (Array[OffsetRange], checkPointDirectory) - */ - implicit class CheckPointers(offsetRangesAndCheckPointDirectory: (Array[OffsetRange], String)) { - /** - * CheckPoints a Tuple of (Array[OffsetRange], checkPointDirectory) - * - * @example (Array(OffsetRange("l1", 11, 1, 1)), "${USER_DEFINED_CHECKPOINT_PATH}").saveCheckPoint - * @return true if Success - * - */ - def saveCheckPoint: Boolean = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - val latestFile = "/latest" - val checkPointDir = offsetRangesAndCheckPointDirectory._2 - val checkPointFile = checkPointDir + latestFile - val contentToWrite = offsetRangesAndCheckPointDirectory._1.toStringOfKafkaOffsetRanges - try { - val conf = new org.apache.hadoop.conf.Configuration() - val fs = FileSystem.get(conf) - val latestHDFSPath = new Path(checkPointFile) - if (!fs.exists(latestHDFSPath)) { - writeHDFSFile(checkPointFile, contentToWrite) - } else { - val timeStamp = System.currentTimeMillis - val toRenameLatestPath = checkPointDir + s"/$timeStamp" - val toRenameLatestPathHDFS = new Path(toRenameLatestPath) - fs.rename(latestHDFSPath, toRenameLatestPathHDFS) - writeHDFSFile(checkPointFile, contentToWrite) - } - } catch { - case ex: Throwable => - throw ex - } - true - } - } - - - /** - * @param checkPointDirectoryPath A Tuple of (Array[OffsetRange], checkPointDirectory) - */ - implicit class CheckPointFetcher(checkPointDirectoryPath: String) { - /** - * Fetches CheckPoints as An Array[OffsetRange] - * - * @example ("USER_DEFINED_CHECKPOINT_PATH").fetchCheckPoint - * @return Some(Array[OffsetRange]) - * - */ - def fetchCheckPoint: Option[Array[OffsetRange]] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - if (checkPointDirectoryPath.isEmpty) throw new HDFSCheckPointerException("Expected CheckPoint Directory, but got Empty String !") - val latestFile = "/latest" - val checkPointDir = checkPointDirectoryPath - val checkPointFile = checkPointDir + latestFile - val conf = new org.apache.hadoop.conf.Configuration() - val fs = FileSystem.get(conf) - val latestHDFSPath = new Path(checkPointFile) - if (fs.exists(latestHDFSPath)) { - val checkPointString = readHDFSFile(checkPointDirectoryPath + "/latest") - println("inside fetchCheckPoint ->" + checkPointString) - Some(checkPointString.split('|').map(x => CheckPointString(x)).toKafkaOffsetRanges) - } else { - None - } - } - } - -} - -/** - * Custom Exception - * - * @param message Message to Throw - * @param cause A Throwable Cause - */ -private class HDFSCheckPointerException(message: String, cause: Throwable) - extends RuntimeException(message) { - if (cause != null) { - initCause(cause) - } - - def this(message: String) = this(message, null) -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/ImplicitKafkaConverters.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/ImplicitKafkaConverters.scala deleted file mode 100644 index bda8c51d..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/ImplicitKafkaConverters.scala +++ /dev/null @@ -1,343 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.utilities - -import java.{lang, util} -import java.util.{Collections, Properties} - -import scala.collection.mutable.ArrayBuffer -import scala.language.implicitConversions - -import org.apache.kafka.clients.admin.AdminClient -import org.apache.kafka.common.TopicPartition -import org.apache.spark.streaming.kafka010.OffsetRange - -import com.paypal.gimel.logger.Logger - -/** - * Case Class to Represent a CheckPoint String. Example "flights,1,1,100" - * - * @param checkPoint - */ -case class CheckPointString(checkPoint: String) - -/** - * Case Class to Represent Brokers and Topics - * - * @param brokers Example : kafka_broker_ip:8081 - * @param topic Example : flights - */ - -case class BrokersAndTopic(brokers: String, topic: String) - -/** - * Provides a set of Implicit , Convenience APIs for developers to use - */ - -object ImplicitKafkaConverters { - - val logger: Logger = Logger() - - /** - * @param offsetRanges An Array of OffsetRange - */ - implicit class OffsetsConverter(offsetRanges: Array[OffsetRange]) { - - /** - * Converts An Array OffsetRange to String of [CheckPoints (comma-separated)], each checkpoint Separated by Pipe - * - * @example Array(OffsetRange("test", 0, 1, 100),OffsetRange("test", 1, 1, 100)).toStringOfKafkaOffsetRanges - * @return String of [CheckPoints (comma-separated)], each checkpoint Separated by Pipe - */ - def toStringOfKafkaOffsetRanges: String = { - offsetRanges.map(offsetRange => offsetRange.toStringOfKafkaOffsetRange).mkString("|") - } - } - - - /** - * @param offsetRange A Kafka OffsetRange - */ - implicit class OffsetConverter(offsetRange: OffsetRange) { - /** - * Converts a Kafka OffsetRange to A CheckPoint (comma-separated) - * - * @return A CheckPoint (comma-separated) - * @example "test,0,0,4".toKafkaOffsetRanges - */ - def toStringOfKafkaOffsetRange: String = { - offsetRange.topic + "," + offsetRange.partition + "," + offsetRange.fromOffset + "," + offsetRange.untilOffset - } - } - - /** - * @param checkPointString A CheckPoint (comma-separated) - */ - implicit class CheckPointConverter(checkPointString: CheckPointString) { - /** - * Converts A CheckPoint (comma-separated) to An OffsetRange - * - * @return An OffsetRange - * @example "test,0,0,4".toKafkaOffsetRanges - */ - def toKafkaOffsetRange: OffsetRange = { - val splitString = checkPointString.checkPoint.split(",") - OffsetRange(splitString(0), splitString(1).toInt, splitString(2).toLong, splitString(3).toLong) - } - } - - /** - * @param checkPointsString an Array of CheckPoints (comma-separated) - */ - implicit class CheckPointsConverter(checkPointsString: Array[CheckPointString]) { - /** - * Converts an Array of CheckPoints (comma-separated) to An Array of OffsetRange - * - * @return An Array of OffsetRange - * @example "test,0,0,4|test,1,0,5".split("|").toKafkaOffsetRanges - */ - def toKafkaOffsetRanges: Array[OffsetRange] = { - checkPointsString.map(eachOffsetString => eachOffsetString.toKafkaOffsetRange) - } - } - - - /** - * @param brokersAndTopic A Tuple of (Comma-Separated Hosts, TopicString) - */ - implicit class TopicPartitionsConverter(brokersAndTopic: BrokersAndTopic) { - - val clientID: Int = scala.util.Random.nextLong().toInt - val brokers: Array[String] = brokersAndTopic.brokers.split(",") - val host1: String = brokers(0).split(":")(0) - val port1: Int = brokers(0).split(":")(1).toInt - val latestTime: Long = -1L - val earliestTime: Long = -2L - - /** - * Converts a given Tuple of KafkaBrokers & Topic into KafkaTopicAndPartitions - * - * @example val testing: Array[TopicAndPartition] = ("localhost:8080,localhost:8081", "test").toTopicAndPartitions - * @return Array[TopicAndPartition] - */ - def toTopicAndPartitions: Map[TopicPartition, (String, Int)] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - val client = AdminClient.create(KafkaUtilities.getDefaultConsumerPropertiesPerBroker(brokersAndTopic.brokers)) - import scala.collection.JavaConverters._ - try { - client.describeTopics( - Collections.singletonList(brokersAndTopic.topic) - ).all().get().asScala.flatMap { topicMetadata => { - topicMetadata._2.partitions().asScala.map { - partitionMetadata => - partitionMetadata.isr() - (new TopicPartition(topicMetadata._1, partitionMetadata.partition()), - (partitionMetadata.leader().host(), partitionMetadata.leader().port())) - } - } - }.toMap - } finally { - client.close() - } - - } - - /** - * Converts a given Tuple of KafkaBrokers & Topic into Array[OffsetRange] available currently in Kafka Cluster - * - * @example val kafkaOffsets:Array[OffsetRange] = ("localhost:8080,localhost:8081", "test").toKafkaOffsetsPerPartition - * @return Array[OffsetRange] - * - */ - def toKafkaOffsetsPerPartition: Array[OffsetRange] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - logger.info(" @Begin --> " + MethodName) - - val topicAndPartitions: Map[TopicPartition, (String, Int)] = brokersAndTopic.toTopicAndPartitions - import scala.collection.JavaConverters._ - val partitions = topicAndPartitions.keySet.asJava - - logger.info("The Topic And Partitions are --> ") - topicAndPartitions.foreach(println) - - val kafkaConsumer = KafkaUtilities.getKafkaConsumer(Some( - KafkaUtilities.getDefaultConsumerPropertiesPerBroker(brokersAndTopic.brokers) - )) - try { - val beginningOffsets: util.Map[TopicPartition, lang.Long] = kafkaConsumer.beginningOffsets(partitions) - val endOffsets: util.Map[TopicPartition, lang.Long] = kafkaConsumer.endOffsets(partitions) - topicAndPartitions.map { - topicAndPartition => - OffsetRange(topicAndPartition._1.topic, topicAndPartition._1.partition, - beginningOffsets.get(topicAndPartition._1), endOffsets.get(topicAndPartition._1)) - }.toArray - } finally { - kafkaConsumer.close() - } - } - - /** - * Take a TopicAndPartition and Returns a Tuple of leader Host & Port - * - * @param topicAndPartition Kafka TopicAndPartition - * @return Tuple(host, port) - */ - private def findLeader(topicAndPartition: (TopicPartition, (String, Int))): (String, Int) = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - val leaderDetails: (String, Int) = (topicAndPartition._2._1, topicAndPartition._2._2) - leaderDetails - } - } - - - /** - * @param offsetRangePairs an Array of Tuple(OffsetRange, OffsetRange). LeftSide Should be Lower Than RightSize - */ - implicit class NewOffsetRangesProvider(offsetRangePairs: (Array[OffsetRange], Array[OffsetRange])) { - /** - * Calculates the New Range of Offsets to Read from Kafka based on a Pair of OffsetRange - * - * @return Array[OffsetRange] - * @example (Array(OffsetRange("a", 0, 1, 1), OffsetRange("a", 1, 2, 100)) ,Array( OffsetRange("a", 1, 2, 100),OffsetRange("a", 0, 1, 100))).toNewOffsetRange - */ - def toNewOffsetRanges: Array[OffsetRange] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - val sortedLeft = offsetRangePairs._1.sortBy(offsetRange => offsetRange.partition) - val sortedRight = offsetRangePairs._2.sortBy(offsetRange => offsetRange.partition) - val combinedAfterSort = sortedLeft.zip(sortedRight) - combinedAfterSort.map { eachPair => - val left = eachPair._1 - val right = eachPair._2 - if (left.topic != right.topic) throw new KafkaOperationsException(s"Invalid Operation ! Seems we are comparing two different topics --> ${left.topic} <> ${right.topic} ") - if (left.untilOffset > right.untilOffset) throw new KafkaOperationsException(s"Left Side Until:Offset ${left.untilOffset} is Higher than Right Side Until:Offset ${right.untilOffset}") - if (left.fromOffset > right.untilOffset) throw new KafkaOperationsException(s"Left Side from:Offset ${left.fromOffset} is Already Beyond Right Side Until:Offset ${right.untilOffset}") - if (left.untilOffset < right.fromOffset) throw new KafkaOperationsException(s"Left Side from:Offset ${left.untilOffset} is Lower Than Right Side from:Offset ${right.untilOffset}. This usually indicates Data Loss !") - val fromOffset = { - if (left.untilOffset == right.untilOffset) { - right.untilOffset - } else { - left.untilOffset - } - } - OffsetRange(left.topic, left.partition, fromOffset, right.untilOffset) - } - } - } - - /** - * @param offsetRanges An Array of OffsetRange - */ - implicit class OffsetRangeRestriction(offsetRanges: Array[OffsetRange]) { - /** - * Limits the OffsetRanges to the given threshold per partition - * - * @example val kafkaOffsets:Array[OffsetRange] = Array(OffsetRange(("localhost:8080,localhost:8081", "test"))).applyThresholdPerPartition(100) - * @return Array[OffsetRange] - * - */ - def applyThresholdPerPartition(maxPerPartition: Long): Array[OffsetRange] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - offsetRanges.map { - eachOffsetRange => - val fromOffset = eachOffsetRange.fromOffset - val maxUntil = fromOffset + maxPerPartition - val untilOffset = eachOffsetRange.untilOffset - val newUntilOffset = scala.math.min(untilOffset, maxUntil) - OffsetRange(eachOffsetRange.topic, eachOffsetRange.partition, eachOffsetRange.fromOffset, newUntilOffset) - } - } - - /** - * Parallelizes an Array of Offset Range, by applying parallelism factor on each Offset Range - * - * @param parallelism Number of parallel shards - * @return Array[OffsetRange] - */ - def parallelizeOffsetRanges(parallelism: Int, minRowsPerParallel: Long): Array[OffsetRange] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - val returningRanges = offsetRanges.flatMap(erange => parallelizeOffsetRange(erange, parallelism, minRowsPerParallel)) - logger.info("Outgoing Array of OffsetRanges --> ") - returningRanges.foreach(logger.info(_)) - returningRanges - } - - // parallelizeOffsetRange(OffsetRange("a", 1, 1, 20), 3) - private def parallelizeOffsetRange(eachRange: OffsetRange, parallel: Int, minRowsPerParallel: Long): Array[OffsetRange] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - val total = eachRange.untilOffset - eachRange.fromOffset - if ((total > minRowsPerParallel)) { - logger.info(s"Incoming Range --> $eachRange") - logger.info(s"Parallel Factor --> $parallel") - val returningRange: scala.collection.mutable.ArrayBuffer[OffsetRange] = ArrayBuffer() - - val recordsPer = scala.math.max(total / parallel, minRowsPerParallel) - var cntr = eachRange.fromOffset - val end = eachRange.untilOffset - while (cntr < end) { - returningRange.append(OffsetRange(eachRange.topic, eachRange.partition, cntr, cntr + recordsPer)) - cntr = cntr + recordsPer - if (cntr + recordsPer > end) { - returningRange.append(OffsetRange(eachRange.topic, eachRange.partition, cntr, end)) - cntr = end - } - } - logger.info("Parallelized Ranges for the given OffsetRange ..") - returningRange.foreach(logger.info(_)) - returningRange.toArray - } else { - logger.info(s"Not Applying Parallelism as the total rows : $total in this Offset Range < min rows per parallel : $minRowsPerParallel ") - Array(eachRange) - } - } - } - -} - -/** - * Custom Exception - * - * @param message Message to Throw - * @param cause A Throwable Cause - */ -private class KafkaOperationsException(message: String, cause: Throwable) - extends RuntimeException(message) { - if (cause != null) { - initCause(cause) - } - - def this(message: String) = this(message, null) -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/ImplicitZKCheckPointers.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/ImplicitZKCheckPointers.scala deleted file mode 100644 index ba2d3d08..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/ImplicitZKCheckPointers.scala +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.utilities - -import scala.language.implicitConversions - -import org.apache.spark.streaming.kafka010.OffsetRange - -import com.paypal.gimel.common.storageadmin.ZooKeeperAdminClient._ -import com.paypal.gimel.kafka.utilities.ImplicitKafkaConverters._ -import com.paypal.gimel.logger.Logger - -case class ZooKeeperHostAndNodes(host: String, nodes: Seq[String]) - -/** - * Provides Implicit, Convenience Functions for Developers to Do CheckPointing Operations - */ -object ImplicitZKCheckPointers { - - val logger = Logger() - - /** - * @param checkPointingInfo Tuple of (ZooKeeperHostAndNode, Array[Kafka OffsetRange]) - */ - implicit class ZKCheckPointers(checkPointingInfo: (ZooKeeperHostAndNodes, Array[OffsetRange])) { - /** - * CheckPoints a Tuple of (Array[OffsetRange], checkPointDirectory) - * - * @example (Array(OffsetRange("l1", 11, 1, 1)),"${USER_DEFINED_CHECKPOINT_PATH}").saveCheckPoint - * @return true if Success - * - */ - def saveZkCheckPoint: Boolean = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - val zkServers = checkPointingInfo._1.host - val zkNodes = checkPointingInfo._1.nodes - val contentToWrite = checkPointingInfo._2.toStringOfKafkaOffsetRanges - try { - zkNodes.map { zkNode => - writetoZK(zkServers, zkNode, contentToWrite) - } - } catch { - case ex: Throwable => - throw ex - } - true - } - - } - - - /** - * @param zooKeeperDetails ZooKeeperHostAndNode - */ - implicit class ZKCheckPointFetcher(zooKeeperDetails: ZooKeeperHostAndNodes) { - /** - * Fetches CheckPoints as An Array[OffsetRange] - * - * @example ("${USER_DEFINED_CHECKPOINT_PATH}").fetchCheckPoint - * @return Some(Array[OffsetRange]) - * - */ - def fetchZkCheckPoint: Option[Array[OffsetRange]] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - logger.info(" @Begin --> " + MethodName) - val zkServers = zooKeeperDetails.host - val zkNodes = zooKeeperDetails.nodes - if (zkServers.isEmpty) throw new ZooKeeperCheckPointerException("Expected CheckPoint Directory, but got Empty String !") - val zkCheckPoints = zkNodes.flatMap { zkNode => - val checkPointString: Option[String] = readFromZK(zkServers, zkNode) - checkPointString match { - case None => - None - case _: Option[String] => - checkPointString.get.split('|').map(x => CheckPointString(x)).toKafkaOffsetRanges - } - }.filter { - None => true - }.toArray - if (zkCheckPoints.isEmpty) { - None - } - else { - Some(zkCheckPoints) - } - } - - /** - * Deletes a ZooKeeper CheckPoint - */ - def deleteZkCheckPoint(): Unit = { - logger.warning(s"WARNING !!!!! Deleting --> host : ${zooKeeperDetails.host} | node : ${zooKeeperDetails.nodes}") - try { - zooKeeperDetails.nodes.map { node => - deleteNodeOnZK(zooKeeperDetails.host, node) - } - } catch { - case ex: Throwable => - throw ex - } - } - } - -} - -/** - * Custom Exception - * - * @param message Message to Throw - * @param cause A Throwable Cause - */ -private class ZooKeeperCheckPointerException(message: String, cause: Throwable) - extends RuntimeException(message) { - if (cause != null) { - initCause(cause) - } - - def this(message: String) = this(message, null) -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/KafkaUtilities.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/KafkaUtilities.scala deleted file mode 100644 index eccbb3c2..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/utilities/KafkaUtilities.scala +++ /dev/null @@ -1,1018 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.utilities - -import java.io.{Closeable, Serializable} -import java.nio.ByteBuffer -import java.util.{Properties, UUID} - -import scala.collection.JavaConverters._ -import scala.collection.immutable.Map -import scala.collection.mutable -import scala.language.implicitConversions -import scala.reflect.runtime.universe._ -import scala.util.parsing.json.JSON - -import org.apache.avro.generic.GenericRecord -import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer} -import org.apache.kafka.clients.producer.ProducerConfig -import org.apache.kafka.common.serialization._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql._ -import org.apache.spark.sql.streaming.StreamingQueryListener -import org.apache.spark.sql.streaming.StreamingQueryListener.{QueryProgressEvent, QueryStartedEvent, QueryTerminatedEvent} -import org.apache.spark.streaming.kafka010._ -import org.apache.spark.streaming.kafka010.KafkaUtils._ -import spray.json._ -import spray.json.DefaultJsonProtocol._ - -import com.paypal.gimel.common.catalog.CatalogProvider -import com.paypal.gimel.common.conf.GimelConstants -import com.paypal.gimel.common.schema.ConfluentSchemaRegistry -import com.paypal.gimel.common.storageadmin -import com.paypal.gimel.common.storageadmin.KafkaAdminUtils -import com.paypal.gimel.common.utilities.DataSetUtils._ -import com.paypal.gimel.datastreamfactory.{StreamCheckPointHolder, WrappedData} -import com.paypal.gimel.kafka.avro.SparkAvroUtilities._ -import com.paypal.gimel.kafka.conf._ -import com.paypal.gimel.kafka.conf.KafkaJsonProtocol.{offsetPropertiesFormat, offsetRangePropertiesFormat} -import com.paypal.gimel.kafka.utilities.ImplicitKafkaConverters._ -import com.paypal.gimel.kafka.utilities.ImplicitZKCheckPointers._ - - -case class MessageInfo[T: TypeTag](key: String, message: T, topic: String, partition: Int, offset: Long) - -/* -Case classes for reading custom offset properties from the user defined properties - */ -case class OffsetRangeProperties(partition: Int, - from: Long, - to: Option[Long]) - -case class OffsetProperties(topic: String, - offsetRange: Array[OffsetRangeProperties]) - -object KafkaUtilities { - - val logger = com.paypal.gimel.logger.Logger() - - /** - * This is a Map of Properties that will be used to set the batch parameters - * , based on the incoming volume of data & user supplied parameters - */ - val defaultRowsPerBatch: Map[Int, Map[String, String]] = Map( - 100000000 -> Map( - KafkaConfigs.batchFetchSize -> "500" - , KafkaConfigs.maxRecordsPerPartition -> "100000000" - , KafkaConfigs.minRowsPerParallelKey -> "100000" - ) - , 50000000 -> Map( - KafkaConfigs.batchFetchSize -> "500" - , KafkaConfigs.maxRecordsPerPartition -> "50000000" - , KafkaConfigs.minRowsPerParallelKey -> "100000" - ) - , 25000000 -> Map( - KafkaConfigs.batchFetchSize -> "250" - , KafkaConfigs.maxRecordsPerPartition -> "25000000" - , KafkaConfigs.minRowsPerParallelKey -> "100000" - ) - , 10000000 -> Map( - KafkaConfigs.batchFetchSize -> "100" - , KafkaConfigs.maxRecordsPerPartition -> "10000000" - , KafkaConfigs.minRowsPerParallelKey -> "100000" - ) - , 1000000 -> Map( - KafkaConfigs.batchFetchSize -> "20" - , KafkaConfigs.maxRecordsPerPartition -> "1000000" - , KafkaConfigs.minRowsPerParallelKey -> "100000" - ) - , 100000 -> Map( - KafkaConfigs.batchFetchSize -> "10" - , KafkaConfigs.maxRecordsPerPartition -> "100000" - , KafkaConfigs.minRowsPerParallelKey -> "10000" - ) - , 30000 -> Map( - KafkaConfigs.batchFetchSize -> "10" - , KafkaConfigs.maxRecordsPerPartition -> "100000" - , KafkaConfigs.minRowsPerParallelKey -> "10000" - ) - ) - - - /** - * Determines whether an incoming volume of messages - * from Kafka is Streamable with given parameters. - * - * @param sparkSession : SparkSession - * @param props Properties - * @param rowsInBatch RowsPerBatch Map - * @return true if data is within streaming capacity - * , false if we need to switch to batch - */ - def isStreamable(sparkSession: SparkSession, props: Map[String, String] - , rowsInBatch: Map[Int, Map[String, String]] = defaultRowsPerBatch): Boolean = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - // val dSet = com.paypal.gimel.DataSet(hiveContext) - val dataSet = props(GimelConstants.DATASET) - // import com.paypal.gimel.DataSetUtils._ - // This is the DataSet Properties - val datasetProps = CatalogProvider.getDataSetProperties(dataSet) - logger.info( - s"""DataSet Props --> - |${datasetProps.props.map(x => s"${x._1} --> ${x._2}").mkString("\n")}""".stripMargin) - val newProps: Map[String, Any] = getProps(props) ++ Map( - GimelConstants.DATASET_PROPS -> datasetProps, - GimelConstants.DATASET -> dataSet, - GimelConstants.RESOLVED_HIVE_TABLE -> resolveDataSetName(dataSet), - GimelConstants.APP_TAG -> getAppTag(sparkSession.sparkContext)) - val conf = new KafkaClientConfiguration(newProps) - logger.info(s"Zookeeper Details --> ${conf.zkHostAndPort} | ${conf.zkCheckPoints}") - val thresholdRows = 1000000000 - val lastCheckPoint: Option[Array[OffsetRange]] = getLastCheckPointFromZK(conf.zkHostAndPort - , conf.zkCheckPoints) - val availableOffsetRange: Array[OffsetRange] = { - BrokersAndTopic(conf.kafkaHostsAndPort, conf.kafkaTopics).toKafkaOffsetsPerPartition - } - if (lastCheckPoint.isDefined) { - logger.info(s"Offsets in CheckPoint --> ${lastCheckPoint.get.mkString("\n")}") - } - logger.info(s"Offsets in Kafka --> ${availableOffsetRange.mkString("\n")}") - val newOffsetRangesForReader: Array[OffsetRange] = { - getNewOffsetRangeForReader(lastCheckPoint, availableOffsetRange, thresholdRows) - } - logger.info(s"New Offsets to Fetch --> ${newOffsetRangesForReader.mkString("\n")}") - val totalMessages = newOffsetRangesForReader.map(oR => oR.untilOffset - oR.fromOffset).sum.toInt - logger.info(s"Total Messages from New Offsets to Fetch --> $totalMessages") - val userSuppliedMaxRows = { - sparkSession.conf.get(KafkaConfigs.rowCountOnFirstRunKey, totalMessages.toString) - } - val totalRows = if (lastCheckPoint.isEmpty) userSuppliedMaxRows.toInt else totalMessages - logger.info(s"Final Total Messages to Fetch --> $totalRows") - val streamCutOff = sparkSession.conf.get(KafkaConfigs.streamCutOffThresholdKey, "100000").toInt - val (batchProps, isStreamable) = totalRows match { - case n if 50000000 <= n => - (rowsInBatch(100000000), false) - case n if 25000000 <= n => - (rowsInBatch(50000000), false) - case n if 10000000 <= n => - (rowsInBatch(25000000), false) - case n if 1000000 <= n => - (rowsInBatch(10000000), false) - case n if streamCutOff <= n => - (rowsInBatch(1000000), false) - case _ => - (Map(), true) - } - logger.info(s"Batch Props --> $batchProps") - val resolvedProps = props ++ batchProps - logger.info(s"Resolved Props --> $resolvedProps") - logger.info(s"isStreamable --> $isStreamable") - resolvedProps.foreach(p => sparkSession.conf.set(p._1, p._2.toString)) - isStreamable - } - - /** - * Convenience Function to checkpoint a given OffsetRange - * - * @param zkHost Host Server for Zookeeper - * @param zkNodes Node where we want to checkPoint - * @param offsetRange Array[OffsetRange] - * @return Boolean indicating checkpointing status - */ - - def inStreamCheckPoint(zkHost: String, zkNodes: Seq[String] - , offsetRange: Array[OffsetRange]): Boolean = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - val zk = ZooKeeperHostAndNodes(zkHost, zkNodes) - (zk, offsetRange).saveZkCheckPoint - } - - /** - * Convenience Function to checkpoint a given OffsetRange - * - * @param sparkSession Spark Session - * @param zkHost Host Server for Zookeeper - * @param zkNodes Node where we want to checkPoint - * @return Boolean indicating checkpointing status - */ - - def inStructuredStreamCheckPoint(sparkSession: SparkSession, zkHost: String, zkNodes: Seq[String]): Unit = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - sparkSession.streams.addListener(new StreamingQueryListener() { - override def onQueryStarted(event: QueryStartedEvent): Unit = Unit - override def onQueryProgress(event: QueryProgressEvent): Unit = { - val queryStatusMap = JSON.parseFull(event.progress.json).get.asInstanceOf[Map[String, Any]] - val endOffsetsMap: Map[String, Map[Any, Any]] = queryStatusMap.get("sources").head.asInstanceOf[List[Any]].head.asInstanceOf[Map[Any, Any]].get("endOffset").head.asInstanceOf[Map[String, Map[Any, Any]]] - val endOffsets = endOffsetsMap.flatMap { x => - x._2.map { y => - OffsetRange(topic = x._1, partition = y._1.asInstanceOf[String].toInt, fromOffset = 0L, untilOffset = y._2.asInstanceOf[Double].longValue()) - } - }.toArray - StreamCheckPointHolder().setCurentCheckPoint(endOffsets) - inStreamCheckPoint(zkHost, zkNodes, endOffsets) - } - override def onQueryTerminated(event: QueryTerminatedEvent): Unit = { - sparkSession.streams.removeListener(this) - } - }) - } - - /** - * Gets the Appropriate Serializer Class - * - * @param serializerClassName Name of the Serializer Class - * @return Serializer Class - */ - - def getSerializer(serializerClassName: String) - : Class[_ >: StringSerializer with ByteArraySerializer <: Serializer[_ >: String with Array[Byte]]] = { - serializerClassName match { - case "org.apache.kafka.common.serialization.StringSerializer" => { - classOf[org.apache.kafka.common.serialization.StringSerializer] - } - case "org.apache.kafka.common.serialization.ByteArraySerializer" => { - classOf[org.apache.kafka.common.serialization.ByteArraySerializer] - } - case _ => { - throw new Exception(s"UnSupported Serializer Class Requested : ${serializerClassName}") - } - } - } - - /** - * Gets the Appropriate DeSerializer Class - * - * @param deserializerClassName Name of the DeSerializer Class - * @return DeSerializer Class - */ - - def getDeserializer(deserializerClassName: String) - : Class[_ >: StringDeserializer with ByteArrayDeserializer <: Deserializer[_ >: String with Array[Byte]]] = { - deserializerClassName match { - case "org.apache.kafka.common.serialization.StringDeserializer" => { - classOf[org.apache.kafka.common.serialization.StringDeserializer] - } - case "org.apache.kafka.common.serialization.ByteArrayDeserializer" => { - classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer] - } - case _ => { - throw new Exception(s"UnSupported DeSerializer Class Requested : ${deserializerClassName}") - } - } - } - - /** - * Gets the Appropriate De/Serializer Class - * - * @param serDe Name of the De/Serializer Class - * @return De/Serializer Class - */ - - def getSerDe(serDe: String): Class[_ >: StringDeserializer - with ByteArrayDeserializer with StringSerializer with ByteArraySerializer <: Closeable] = { - serDe match { - case "org.apache.kafka.common.serialization.StringDeserializer" => { - classOf[org.apache.kafka.common.serialization.StringDeserializer] - } - case "org.apache.kafka.common.serialization.ByteArrayDeserializer" => { - classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer] - } - case "org.apache.kafka.common.serialization.StringSerializer" => { - classOf[org.apache.kafka.common.serialization.StringSerializer] - } - case "org.apache.kafka.common.serialization.ByteArraySerializer" => { - classOf[org.apache.kafka.common.serialization.ByteArraySerializer] - } - case _ => { - throw new Exception(s"UnSupported serDe Class Requested : ${serDe}") - } - } - } - - /** - * Converts RDD[WrappedData] to DataFrame with just 1 column - - * which is the entire message String from Kafka - * - * @param sqlContext SQLContext - * @param columnAlias Name of Column in DataFrame - * @param wrappedData WrappedData - * @return DataFrame - */ - def wrappedStringDataToDF(columnAlias: String, sqlContext: SQLContext - , wrappedData: RDD[WrappedData]): DataFrame = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - logger.info("Attempting to Convert Value in Wrapped Data to String Type") - try { - val rdd: RDD[(String, String)] = wrappedData.map { x => - (x.key.asInstanceOf[String], x.value.asInstanceOf[String]) - } - val df = rddAsDF(sqlContext, columnAlias, rdd) - logger.info("Completed --> Convert Value to String Type") - df - } catch { - case ex: Throwable => - ex.printStackTrace() - throw ex - } - - } - - /** - * Completely Clear the CheckPointed Offsets, leading to Read from Earliest offsets from Kafka - * - * @param zkHost Zookeeper Host - * @param zkNodes Zookeeper Path - * @param msg Some Message or A Reason for Clearing CheckPoint - */ - def clearCheckPoint(zkHost: String, zkNodes: Seq[String], msg: String): Unit = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - val zk = ZooKeeperHostAndNodes(zkHost, zkNodes) - zk.deleteZkCheckPoint() - } - - - /** - * Gets the Latest CheckPoint from Zookeeper, if available - * - * @param zkHost Host Server for Zookeeper - * @param zkNodes Node where we want to checkPoint - * @return Option[Array[OffsetRange] - */ - - def getLastCheckPointFromZK(zkHost: String, zkNodes: Seq[String]): Option[Array[OffsetRange]] = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - logger.info(" @Begin --> " + MethodName) - - try { - val zk = ZooKeeperHostAndNodes(zkHost, zkNodes) - val lastCheckPoint: Option[Array[OffsetRange]] = zk.fetchZkCheckPoint - lastCheckPoint - } catch { - case ex: Throwable => - ex.printStackTrace() - throw ex - } - } - - /** - * Function Gets - * Either : The difference between lastCheckPoint & latestOffsetRange - * Or : latestOffsetRange from Kafka - * - * @param lastCheckPoint savedCheckPoint, if available - * @param availableOffsetRange latestOfffsetRange from Kafka - * @param fetchRowsOnFirstRun This will be used if reading from kafka without - * any prior checkpoint, - * to ensure we read only last N messages - * from topic as requested by client - * @return Array[OffsetRange] - */ - - def getNewOffsetRangeForReader(lastCheckPoint: Option[Array[OffsetRange]] - , availableOffsetRange: Array[OffsetRange] - , fetchRowsOnFirstRun: Long): Array[OffsetRange] = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - try { - val newOffsetRangesForReader = lastCheckPoint match { - case None => { - logger.warning( - s"""No CheckPoint Found. - |Reader will attempt to fetch "from beginning" From Kafka !""".stripMargin) - availableOffsetRange.map { - eachOffsetRange => - val fromOffset = scala.math.min(fetchRowsOnFirstRun - , eachOffsetRange.untilOffset - eachOffsetRange.fromOffset) - logger.info(s"Since this is first run," + - s" will try to fetch only ${fromOffset} rows from Kafka") - OffsetRange(eachOffsetRange.topic, eachOffsetRange.partition - , eachOffsetRange.untilOffset - fromOffset, eachOffsetRange.untilOffset) - } - } - case Some(lastCheckPoint) => { - logger.info("""Found CheckPoint """) - (lastCheckPoint, availableOffsetRange).toNewOffsetRanges - } - } - newOffsetRangesForReader - } - catch { - case ex: Throwable => { - ex.printStackTrace() - throw ex - } - } - } - - /** - * Function Gets - * a custom offset range as a JSON from the user defined properties - * Converts it to an array of offset ranges and returns them - * - * @param kafkaTopics sequence of topics - * @param offsetRange user given custom offset ranges, if available - * @return Array[OffsetRange] - */ - - def getCustomOffsetRangeForReader(kafkaTopics: Seq[String], offsetRange: String, consumerMode: String): Array[OffsetRange] = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - try { - val offsetRangeObject = offsetRange.parseJson.convertTo[Seq[OffsetProperties]] - val finalOffsetRanges = offsetRangeObject.flatMap { - eachTopicRange => - eachTopicRange.offsetRange.map { - eachOffsetRange => { - var toOffset = 0L - if (consumerMode == KafkaConstants.gimelAuditRunTypeStream) { - toOffset = eachOffsetRange.to.getOrElse(-1) - } - else if (consumerMode == KafkaConstants.gimelAuditRunTypeBatch) { - toOffset = eachOffsetRange.to.get - } - if(!kafkaTopics.contains(eachTopicRange.topic)) { - throw new Exception("The topic specified in custom offset range does not match the subscribed topic! Please unset the previous value or check your properties") - } - OffsetRange(eachTopicRange.topic, eachOffsetRange.partition, eachOffsetRange.from, toOffset) - } - } - }.toArray - finalOffsetRanges - } catch { - case ex: Throwable => - ex.printStackTrace() - throw ex - } - } - - /** - * Converts an RDD[Wrapped Data] into RDD[GenericRecord] - * - * @param wrappedDataRDD RDD[WrappedData] - * @param avroSchemaKey AvroSchemaKey | Example flights , flights.flights_log - * @param avroSchemaURL Confluent Schema Registry URL:Port - * @param avroSchemaSource Specifies whether schema is inline text or from CDH schema registry - * @param avroSchemaString Avro Schema String for flights - * @param isStreamParallel true indicates : can repartition data for parallelism. - * false is usually set for preserving ordering of data - * as received from kafka - * @param streamParallels Repartition factor, for example : 10 indicates repartition to - * 10 executors - * @return RDD[GenericRecord] - */ - def wrappedDataToAvro(wrappedDataRDD: RDD[WrappedData], avroSchemaKey: String, - avroSchemaURL: String, - avroSchemaSource: String, avroSchemaString: String, - isStreamParallel: Boolean, streamParallels: Int, - cdhAllSchemaDetails: Option[Map[String, - (String, mutable.Map[Int, String])]]): RDD[GenericRecord] = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - try { - val parallelRDD = if (isStreamParallel) { - wrappedDataRDD.repartition(streamParallels) - } else { - wrappedDataRDD - } - val avroRecord: RDD[GenericRecord] = parallelRDD.map { - x => bytesToGenericRecord(x.value.asInstanceOf[Array[Byte]], avroSchemaString) - } - val finalAvroRecord = avroSchemaSource.toUpperCase() match { - case "CDH" => - deserializeCurRec(avroRecord, cdhAllSchemaDetails) - case _ => avroRecord - } - finalAvroRecord - } - catch { - case ex: Throwable => { - ex.printStackTrace() - throw ex - } - } - } - - /** - * Fetches the Schema for each Topic with version - * - * @param schemaSubject Schema Key - * @param avroSchemaURL Confluent Schema URL - * @return Map of Topic -> (Version & Schema) - */ - - def getAllSchemasForSubject(schemaSubject: String, avroSchemaURL: String) - : (String, mutable.Map[Int, String]) = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - val schemaLookup: scala.collection.mutable.Map[Int, String] = scala.collection.mutable.Map() - val schemaRegistryClient = new ConfluentSchemaRegistry(avroSchemaURL) - val k = schemaRegistryClient.getAllVersions(schemaSubject).asScala - val k2 = k.map { eachVersion => - val version = eachVersion.toString.toInt - version -> schemaRegistryClient.getVersion(schemaSubject, version).getSchema - }.toMap - k2.foreach(entry => schemaLookup.put(entry._1, entry._2)) - val latestSchema = schemaRegistryClient.getLatestVersion(schemaSubject).getSchema - (latestSchema, schemaLookup) - } - - - /** - * Deserialize the CDH record (bytes) , get GenericRecord - * - * @param avroRecord Avro GenericRecord RDD - * @param cdhAllSchemaDetails All the Subjects with LatestSchema and EachVersion - * @return Avro GenericRecord RDD - */ - def deserializeCurRec(avroRecord: RDD[GenericRecord] - , cdhAllSchemaDetails: Option[Map[String, - (String, mutable.Map[Int, String])]]): RDD[GenericRecord] = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - val additionalFields = getAdditionalFields() - try { - val actualRecord: RDD[GenericRecord] = avroRecord.map { eachRecord => - val eachRecordSchemaSubject = eachRecord.get("schemaSubject").toString - val schemaThisRec = cdhAllSchemaDetails.get(eachRecordSchemaSubject)._1 - val eachRec: Array[Byte] = eachRecord.get("currentRecord").asInstanceOf[ByteBuffer].array() - var metaColumnsMap = scala.collection.immutable.Map[String, String]() - // Add mandatory meta columns, gg commit timestamp, rba and opType - additionalFields.foreach { - field => metaColumnsMap += (field._1 -> eachRecord.get(field._2).toString) - } - val genericRecord: GenericRecord = bytesToGenericRecord(eachRec, schemaThisRec) - val newSchema = addAdditionalFieldsToSchema(additionalFields.keySet.toList, schemaThisRec) - val newGenericRecord = copyToGenericRecord(genericRecord, schemaThisRec, newSchema) - metaColumnsMap.foreach { kv => newGenericRecord.put(kv._1, kv._2) } - newGenericRecord - } - actualRecord - } - catch { - case ex: Throwable => { - ex.printStackTrace() - throw ex - } - } - } - - /** - * Lists Additional fields to pick from CDH metadata record. - * - * @return List of Metadata columns - */ - def getAdditionalFields(): scala.collection.immutable.Map[String, String] = - scala.collection.immutable.Map("gg_commit_timestamp" -> "opTs" - , "opt_type" -> "opType", "trail_seq_no" -> "trailSeqno", "trail_rba" -> "trailRba") - - - /** - * Adds additional fields to the Avro Schem - * - * @param additionalFields List of fields to Add - * @param schemaString Input Avro Schema - * @return Updated Avro Schema String - */ - def addAdditionalFieldsToSchema(additionalFields: List[String], schemaString: String) - : String = { - // Parse as JsValue - val schemaAsJsVal = schemaString.parseJson - // Convert to JsObject - val schemaAsJsObject = schemaAsJsVal.asJsObject - // Get the Map of each element & Value - val schemaElementsMap: Map[String, JsValue] = schemaAsJsObject.fields - // These fields will be added with "to-add" fields - val schemaFields = schemaAsJsObject.getFields("fields").head.convertTo[Seq[JsValue]] - val additionalFieldsJSON: List[String] = additionalFields.map { - x => s"""{"name":"${x}","type":["null","string"]}""".stripMargin - } // "to-add" fields - val additionalFieldsAsJsVal: List[JsValue] = additionalFieldsJSON.map { x => x.parseJson } - // added both fields - val combinedFields: Seq[JsValue] = schemaFields ++ additionalFieldsAsJsVal - // formation of a String so it can be inferred as JsVal - val combinedFieldsAsString = combinedFields.map { - x => x.asJsObject.compactPrint - }.mkString("[", ",", "]") - val combinedFieldsAsJsValue = combinedFieldsAsString.parseJson - val toOverride = scala.collection.Map("fields" -> combinedFieldsAsJsValue) - val k12 = schemaElementsMap ++ toOverride - k12.toJson.compactPrint - } - - /** - * Get the Column Alias Name for a Given Single Column DF to be read from Kafka Topic - * that has human readable message - * - * @param conf KafkaClientConfiguration - * @return column alias name - */ - def kafkaMessageColumnAlias(conf: KafkaClientConfiguration): String = { - conf.tableProps.getOrElse("kafka.message.column.alias", "message").toString - } - - - /** - * InTakes RDD And Converts to DataFrame - * - * @param sqlContext SQL Context - * @param messageColumnAlias Message Column Name - * @param rdd RDD[(String,String)] - * @return DataFrame - */ - def stringRddAsDF(sqlContext: SQLContext, messageColumnAlias: String - , rdd: RDD[(String, String)]): DataFrame = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - try { - val dataIntermediate = sqlContext.createDataFrame(rdd) - .withColumnRenamed("_2", "message") - .withColumnRenamed("_1", "key") - val df = dataIntermediate.select("message").withColumnRenamed("message", messageColumnAlias) - df - } - catch { - case ex: Throwable => { - ex.printStackTrace() - logger.error(s"Failed While Attempting to Convert RDD to DF") - throw ex - } - } - } - - /** - * Converts RDD[WrappedData] to DataFrame - * - * @param sqlContext SQLContext - * @param valueMessageType Message Type From Kafka - such as string, json, binary.. - * @param keySerializer Key Serializer - * @param valueSerializer Value Serializer - * @param rdd RDD[Wrapped Data] - * @param kafkaValueMessageColAlias Column Alias in DataFrame for Messages from Kafka - * @param avroSchemaString Avro Schema String for Derserialization - * @param avroSchemaSource Avro Schema Source such as Inline or CDH Confluent Schema Registry - * @param cdhTopicSchemaMetadata CDH TopicSchema Details - * @param cdhAllSchemaDetails The Topic , Version, Schema information - * @return DataFrame - */ - - def rddToDF(sqlContext: SQLContext - , valueMessageType: Option[String] - , keySerializer: String - , valueSerializer: String - , rdd: RDD[WrappedData] - , kafkaValueMessageColAlias: String = "value" - , avroSchemaString: String - , avroSchemaSource: String - , cdhTopicSchemaMetadata: Option[String] - , cdhAllSchemaDetails: Option[Map[String, (String, mutable.Map[Int, String])]]) - : DataFrame = { - (valueMessageType, valueSerializer) match { - // Bytes Messages - case (Some("binary"), "org.apache.kafka.common.serialization.ByteArraySerializer") => - val rDD = rdd.map { x => (x.key.asInstanceOf[String], x.value.asInstanceOf[Array[Byte]]) } - // logger.info("Byte Messages -->"); - // rDD.cache.collect.take(10).foreach(x => logger.info(x)) - val columnAlias = kafkaValueMessageColAlias - byteRddAsDF(sqlContext, columnAlias, rDD) - // String Messages - case (Some("string"), "org.apache.kafka.common.serialization.StringSerializer") => - val rDD = rdd.map { x => (x.key.asInstanceOf[String], x.value.asInstanceOf[String]) } - // logger.info("String Messages -->"); - // rDD.cache.collect.take(10).foreach(x => logger.info(x)) - val columnAlias = kafkaValueMessageColAlias - stringRddAsDF(sqlContext, columnAlias, rDD) - // JSON Messages - case (Some("json"), "org.apache.kafka.common.serialization.StringSerializer") => - val rDD: RDD[String] = rdd.map { x => x.value.asInstanceOf[String] } - // logger.info("JSON Messages -->"); - // rDD.cache.collect.take(10).foreach(x => logger.info(x)) - sqlContext.read.json(rDD) - // Avro - CDH | Generic Avro - case (_, "org.apache.kafka.common.serialization.ByteArraySerializer") => - val rDD = rdd.map { x => (x.key, x.value.asInstanceOf[Array[Byte]]) } - // logger.info("Raw Messages -->"); - // rDD.cache.collect.take(10).foreach(x => logger.info(x)) - val avroRecord: RDD[GenericRecord] = rDD.map { x => - bytesToGenericRecord(x._2, avroSchemaString) - } - val (finalAvroRecord, finalSchema) = avroSchemaSource.toUpperCase() match { - case KafkaConstants.gimelKafkaAvroSchemaCDH => { - val newSchemaCDH = addAdditionalFieldsToSchema(getAdditionalFields().keySet.toList - , cdhTopicSchemaMetadata.get) - (deserializeCurRec(avroRecord, cdhAllSchemaDetails), newSchemaCDH) - } - case _ => (avroRecord, avroSchemaString) - } - genericRecordtoDF(sqlContext, finalAvroRecord, finalSchema) - // Other Types - case _ => throw new Exception("Unsupported Configuration or Serialization Techniques") - } - } - - /** - * Returns A Wrapped Message from Kafka - * - * @param sqlContext SQLContext - * @param conf KafkaClientConfiguration - * @param parallelizedRanges Array[OffsetRange] - * @return RDD[WrappedData] - */ - - def getFromKafkaAsWrappedData(sqlContext: SQLContext - , conf: KafkaClientConfiguration - , parallelizedRanges: Array[OffsetRange] - ): RDD[WrappedData] = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - val (avroSchemaString, avroSchemaKey, avroSchemaURL) = (conf.avroSchemaString - , conf.avroSchemaKey - , conf.avroSchemaURL - ) - val kafkaParams: java.util.Map[String, Object] = new java.util.HashMap() - conf.kafkaConsumerProps.foreach { x => kafkaParams.put(x._1, x._2) } - logger.info(s"Final Kafka Params --> ${kafkaParams.asScala.mkString("\n", "\n", "\n")}") - logger.info( - s"""kafka.message.value.type --> ${conf.kafkaMessageValueType} - |\nValue Serializer --> ${conf.kafkaValueSerializer}""".stripMargin - ) - try { - - val rdd: RDD[_ >: (String, Array[Byte]) with (String, String) <: (String, Serializable)] = - (conf.kafkaMessageValueType, conf.kafkaValueSerializer) match { - // Bytes Messages - case (Some("binary"), "org.apache.kafka.common.serialization.ByteArraySerializer") => - val rDDConsumerRec: RDD[ConsumerRecord[String, Array[Byte]]] = - createRDD[String, Array[Byte]]( - sqlContext.sparkContext, kafkaParams - , parallelizedRanges, LocationStrategies.PreferConsistent) - rDDConsumerRec.map { x => (x.key(), x.value()) } - // String Messages - case (Some("string"), "org.apache.kafka.common.serialization.StringSerializer") => - val rDDConsumerRec: RDD[ConsumerRecord[String, String]] = - createRDD[String, String](sqlContext.sparkContext - , kafkaParams, parallelizedRanges, LocationStrategies.PreferConsistent) - rDDConsumerRec.map { x => (x.key(), x.value()) } - // JSON Messages - case (Some("json"), "org.apache.kafka.common.serialization.StringSerializer") => - val rDDConsumerRec: RDD[ConsumerRecord[String, String]] = - createRDD[String, String](sqlContext.sparkContext - , kafkaParams, parallelizedRanges, LocationStrategies.PreferConsistent) - rDDConsumerRec.map { x => (x.key(), x.value()) } - // Avro - CDH | Generic Avro - case (_, "org.apache.kafka.common.serialization.ByteArraySerializer") => - val rDDConsumerRec: RDD[ConsumerRecord[String, Array[Byte]]] = - createRDD[String, Array[Byte]](sqlContext.sparkContext - , kafkaParams, parallelizedRanges, LocationStrategies.PreferConsistent) - rDDConsumerRec.map { x => (x.key(), x.value()) } - // Other Types - case _ => throw new Exception("Unsupported Configuration or Serialization Techniques") - } - - rdd.map(x => WrappedData(x._1, x._2)) - } - catch { - case ex: Throwable => { - ex.printStackTrace() - val messageString = - s"""kafkaParams --> ${kafkaParams.asScala.mkString(" \n ")}""".stripMargin - logger.error(s"Unable to Fetch from Kafka for given parameters --> ${messageString}") - throw ex - } - } - } - - /** - * Returns DataFrame -fetching messages from Kafka - * - * @param sqlContext SQLContext - * @param conf KafkaClientConfiguration - * @param parallelizedRanges Array[OffsetRange] - * @return DataFrame - */ - - def getAsDFFromKafka(sqlContext: SQLContext, conf: KafkaClientConfiguration - , parallelizedRanges: Array[OffsetRange]): DataFrame = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - val kafkaParams: java.util.Map[String, Object] = new java.util.HashMap() - conf.kafkaConsumerProps.foreach { x => kafkaParams.put(x._1, x._2) } - logger.info(s"Final Kafka Params --> ${kafkaParams.asScala.mkString("\n", "\n", "\n")}") - logger.info( - s"""kafka.message.value.type --> ${conf.kafkaMessageValueType} - |\nValue Serializer --> ${conf.kafkaValueSerializer}""".stripMargin) - val wrappedDataRdd: RDD[WrappedData] = getFromKafkaAsWrappedData(sqlContext, conf, parallelizedRanges) - rddToDF(sqlContext, conf.kafkaMessageValueType, conf.kafkaKeySerializer - , conf.kafkaValueSerializer, wrappedDataRdd, "value", conf.avroSchemaString - , conf.avroSchemaSource, conf.cdhTopicSchemaMetadata, conf.cdhAllSchemaDetails) - } - - /** - * Converts Avro RDD to Spark DataFrame - * - * @param avroRecord RDD Generic Record - * @param sqlContext SQLContext - * @param avroSchemaString Avro Schema String - * @param avroSchemaSource Avro Schema Source - * @param cdhTopicSchemaMetadata CDH Topic Metadata Details - * @param cdhAllSchemaDetails CDH Schema Details (Keys, Schemas..) - * @return DataFrame - */ - - @deprecated - def avroToDF1(avroRecord: RDD[GenericRecord] - , sqlContext: SQLContext - , avroSchemaString: String - , avroSchemaSource: String - , cdhTopicSchemaMetadata: Option[String] - , cdhAllSchemaDetails: Option[Map[String, (String, mutable.Map[Int, String])]]) - : DataFrame = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - val (finalAvroRecord, finalSchema) = avroSchemaSource match { - case KafkaConstants.gimelKafkaAvroSchemaCDH => { - val newSchemaCDH = addAdditionalFieldsToSchema(getAdditionalFields().keySet.toList - , cdhTopicSchemaMetadata.get) - (deserializeCurRec(avroRecord, cdhAllSchemaDetails), newSchemaCDH) - } - case _ => (avroRecord, avroSchemaString) - } - val df = genericRecordtoDF(sqlContext, finalAvroRecord, finalSchema) - df - } - - /** - * InTakes RDD And Converts to DataFrame - * - * @param sqlContext SQL Context - * @param messageColumnAlias Message Column Name - * @param rdd RDD[(String, String)] - * @return DataFrame - */ - def rddAsDF(sqlContext: SQLContext, messageColumnAlias: String - , rdd: RDD[(String, String)]): DataFrame = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - try { - val dataIntermediate = sqlContext.createDataFrame(rdd) - .withColumnRenamed("_2", "message").withColumnRenamed("_1", "key") - dataIntermediate.select("message").withColumnRenamed("message", messageColumnAlias) - } catch { - case ex: Throwable => - ex.printStackTrace() - logger.error(s"Failed While Attempting to Convert RDD to DF") - throw ex - } - } - - /** - * InTakes RDD And Converts to DataFrame - * - * @param sqlContext SQL Context - * @param messageColumnAlias Message Column Name - * @param rdd RDD[(String,Array[Byte])] - * @return DataFrame - */ - def byteRddAsDF(sqlContext: SQLContext, messageColumnAlias: String - , rdd: RDD[(String, Array[Byte])]): DataFrame = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - try { - val dataIntermediate = sqlContext.createDataFrame(rdd) - .withColumnRenamed("_2", "message").withColumnRenamed("_1", "key") - dataIntermediate.select("message").withColumnRenamed("message", messageColumnAlias) - } - catch { - case ex: Throwable => { - ex.printStackTrace() - logger.error(s"Failed While Attempting to Convert RDD to DF") - throw ex - } - } - } - - /** - * Creates a Topic in Kafka if it does not exists - * - * @param zookKeeperHostAndPort Zookeeper Host & Port | Example localhost:2181 - * @param kafkaTopicName Kafka Topic Name - * @param numberOfPartitions Number of Partitions - * @param numberOfReplica Number of Replicas - */ - def createTopicIfNotExists(zookKeeperHostAndPort: String, kafkaTopicName: String - , numberOfPartitions: Int, numberOfReplica: Int): Unit = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - KafkaAdminUtils.createTopicIfNotExists( - zookKeeperHostAndPort - , kafkaTopicName - , numberOfPartitions - , numberOfReplica - ) - } - - /** - * Delete a Topic if it exists - * - * @param zookKeeperHostAndPort Zookeeper Host & Port | Example localhost:2181 - * @param kafkaTopicName Kafka Topic Name - */ - def deleteTopicIfExists(zookKeeperHostAndPort: String, kafkaTopicName: String): Unit = { - def MethodName: String = new Exception().getStackTrace.apply(1).getMethodName - - logger.info(" @Begin --> " + MethodName) - - storageadmin.KafkaAdminUtils.deleteTopicIfExists( - zookKeeperHostAndPort - , kafkaTopicName - ) - } - - /** - * - * @param properties - * @return - */ - def getKafkaConsumer(properties: Option[Properties] = None): KafkaConsumer[Object, Object] = { - val consumerProperties = new Properties() - if (properties.isDefined) { - consumerProperties.putAll(properties.get) - } - // Ensure the serializer configuration is set though its not needed - consumerProperties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[BytesDeserializer].getName) - consumerProperties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[BytesDeserializer].getName) - val group = consumerProperties.get(ConsumerConfig.GROUP_ID_CONFIG) - if (group == null) { - consumerProperties.put(ConsumerConfig.GROUP_ID_CONFIG, "kafka-consumer-offset-client-" + UUID.randomUUID) - } - new KafkaConsumer[Object, Object](consumerProperties) - } - - /** - * - * @param broker - * @return - */ - def getDefaultConsumerPropertiesPerBroker(broker: String): Properties = { - val props = new Properties() - props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, broker) - props - } -} - -/** - * Custom Exception for KafkaUtilities related errors - * - * @param message Message to Throw - * @param cause A Throwable Cause - */ -class KafkaUtilitiesException(message: String, cause: Throwable) - extends RuntimeException(message) { - if (cause != null) { - initCause(cause) - } - - def this(message: String) = this(message, null) -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/writer/KafkaBatchProducer.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/writer/KafkaBatchProducer.scala deleted file mode 100644 index bb0c3413..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/main/scala/com/paypal/gimel/kafka/writer/KafkaBatchProducer.scala +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.writer - -import java.util.Properties - -import scala.collection.JavaConverters._ -import scala.language.implicitConversions -import scala.reflect.runtime.universe._ - -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame - -import com.paypal.gimel.kafka.avro.SparkAvroUtilities._ -import com.paypal.gimel.kafka.conf.KafkaClientConfiguration -import com.paypal.gimel.kafka.utilities.KafkaUtilitiesException - -/** - * Implements Produce to Kafka Logic Here - */ -object KafkaBatchProducer { - - val logger = com.paypal.gimel.logger.Logger() - - /** - * InTakes a DataFrame - * Convert to Avro Record - * Serialize the record into Bytes - * Publish to Kafka - * - * @param conf KafkaClientConfiguration - * @param data RDD - */ - def produceToKafka[T: TypeTag](conf: KafkaClientConfiguration, data: RDD[T]): Unit = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - val kafkaProps: Properties = conf.kafkaProducerProps - val kafkaTopic = conf.kafkaTopics - logger.info(s"Kafka Props for Producer -> ${kafkaProps.asScala.mkString("\n")}") - logger.info("Begin Publishing to Kafka....") - try { - data.foreachPartition { eachPartition => - val producer: KafkaProducer[Nothing, T] = new KafkaProducer(kafkaProps) - val resp = eachPartition.map { messageString => - val rec = new ProducerRecord(kafkaTopic, messageString) - producer.send(rec) - } - resp.length - producer.close() - } - } - catch { - case ex: Throwable => { - ex.printStackTrace() - val msg = - s""" - |kafkaTopic -> ${kafkaTopic} - |kafkaParams --> ${kafkaProps.asScala.mkString("\n")}} - """.stripMargin - throw new KafkaUtilitiesException(s"Failed While Pushing Data Into Kafka \n ${msg}") - } - } - logger.info("Publish to Kafka - Completed !") - } - - /** - * InTakes a DataFrame - * Convert to Avro Record - * Serialize the record into Bytes - * Publish to Kafka - * - * @param conf KafkaClientConfiguration - * @param dataFrame DataFrame - */ - def produceToKafka(conf: KafkaClientConfiguration, dataFrame: DataFrame): Unit = { - def MethodName: String = new Exception().getStackTrace().apply(1).getMethodName() - logger.info(" @Begin --> " + MethodName) - - logger.info(s"kafka.message.value.type --> ${conf.kafkaMessageValueType} \nValue Serializer --> ${conf.kafkaValueSerializer}") - (conf.kafkaMessageValueType, conf.kafkaValueSerializer) match { - case (Some("binary"), "org.apache.kafka.common.serialization.ByteArraySerializer") => - val rdd = dataFrame.rdd.map { x => x.getAs[Array[Byte]](0) } - produceToKafka(conf, rdd) - case (Some("string"), "org.apache.kafka.common.serialization.StringSerializer") => - val rdd = dataFrame.rdd.map { x => x.getAs[String](0) } - produceToKafka(conf, rdd) - case (Some("json"), "org.apache.kafka.common.serialization.StringSerializer") => - val rdd = dataFrame.toJSON.rdd - produceToKafka(conf, rdd) - case (_, "org.apache.kafka.common.serialization.ByteArraySerializer") => { - val kafkaProps: Properties = conf.kafkaProducerProps - val avroSchemaString = conf.avroSchemaString - val kafkaTopic = conf.kafkaTopics - logger.debug(s"Kafka Props for Producer -> ${kafkaProps.asScala.mkString("\n")}") - logger.debug(s"avro Schema --> ${avroSchemaString}") - logger.debug(s"dataframe Schema --> ${dataFrame.schema}") - try { - if (!isDFFieldsEqualAvroFields(dataFrame, avroSchemaString)) { - throw new KafkaUtilitiesException(s"Incompatible DataFrame Schema Vs Provided Avro Schema.") - } - val genericRecordRDD = dataFrametoGenericRecord(dataFrame, avroSchemaString) - val serializedRDD: RDD[Array[Byte]] = genericRecordRDD.map(genericRecord => genericRecordToBytes(genericRecord, avroSchemaString)) - logger.info("Begin Publishing to Kafka....") - serializedRDD.foreachPartition { - eachPartition => - val producer: KafkaProducer[Nothing, Array[Byte]] = new KafkaProducer(kafkaProps) - val resp = eachPartition.map { - arrayByte => - val rec = new ProducerRecord(kafkaTopic, arrayByte) - producer.send(rec) - } - resp.length - producer.close() - } - } - catch { - case ex: Throwable => { - ex.printStackTrace() - val msg = - s""" - |kafkaTopic -> ${kafkaTopic} - |kafkaParams --> ${kafkaProps.asScala.mkString("\n")}} - |avroSchemaString --> ${avroSchemaString} - """.stripMargin - throw new KafkaUtilitiesException(s"Failed While Pushing Data Into Kafka \n ${msg}") - } - } - logger.info("Publish to Kafka - Completed !") - } - case _ => throw new Exception(s"UnSupported Serialization --> ${conf.kafkaValueSerializer}") - } - - } -} diff --git a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/test/scala/com/paypal/gimel/kafka/utilities/KafkaConvertersTests.scala b/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/test/scala/com/paypal/gimel/kafka/utilities/KafkaConvertersTests.scala deleted file mode 100644 index f3b698fb..00000000 --- a/gimel-dataapi/gimel-connectors/gimel-kafka-0.10/src/test/scala/com/paypal/gimel/kafka/utilities/KafkaConvertersTests.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Copyright 2018 PayPal Inc. - * - * 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 com.paypal.gimel.kafka.utilities - -import scala.language.implicitConversions - -import org.apache.spark.streaming.kafka010.OffsetRange -import org.scalatest._ - -import com.paypal.gimel.kafka.utilities.ImplicitKafkaConverters._ -import com.paypal.gimel.kafka.utilities.KafkaUtilities._ - -class KafkaConvertersTests extends FunSpec with Matchers { - - it("should convert array of offset ranges to a single parsable string") { - val sampleRange: Array[OffsetRange] = Array( - OffsetRange("test", 0, 1, 100), - OffsetRange("test", 1, 1, 100)) - val stringified = sampleRange.toStringOfKafkaOffsetRanges - stringified shouldBe "test,0,1,100|test,1,1,100" - } - - it("should converr offset Range to a single parsable checkPoint String") { - val sampleRange = OffsetRange("test", 0, 1, 100) - val stringiFied = sampleRange.toStringOfKafkaOffsetRange - stringiFied shouldBe "test,0,1,100" - } - - it("should convert a single parsable CheckPoint string to a valid offset Range") { - val sampleString = "test,0,1,100" - val offsetRange = CheckPointString(sampleString).toKafkaOffsetRange - offsetRange shouldBe OffsetRange("test", 0, 1, 100) - } - - it("should convert composite `CheckPoint (Array[String])` to a valid Array(Offset Range)") { - val expectedOffsetRanges = Array(OffsetRange("test", 0, 1, 100), OffsetRange("test", 1, 1, 101)) - val sampleString: Array[String] = "test,0,1,100|test,1,1,101".split('|') - val offsetRanges: Array[OffsetRange] = sampleString.map(CheckPointString).toKafkaOffsetRanges - offsetRanges shouldEqual expectedOffsetRanges - } - - it("should convert a json string of custom partition information to an array of offset ranges") { - val sampleRange: Array[OffsetRange] = Array( - OffsetRange("test", 0, 1, 100), - OffsetRange("test", 1, 1, 100)) - val defaultRange: Array[OffsetRange] = Array( - OffsetRange("test", 0, 1, 100), - OffsetRange("test", 2, 1, 100)) - val sampleJson: String = - """[{"topic":"test","offsetRange":[{"partition":0,"from":1,"to":100},{"partition":1,"from":1,"to":100}]}]""" - /* - Happy case for Batch - The value returned should be a valid conversion of the sampleJson to an Array[OffsetRange] - */ - val finalOffsetRanges: Array[OffsetRange] = getCustomOffsetRangeForReader("test".split(","), sampleJson, "BATCH") - finalOffsetRanges shouldEqual(sampleRange) - - val sampleRangeForStream: Array[OffsetRange] = Array( - OffsetRange("test", 0, 1, 100), - OffsetRange("test", 1, 1, -1)) - /* - To offset missing case for Stream - The value returned should be a valid conversion of the sampleJson to an Array[OffsetRange] with To offset as -1 - */ - val sampleJsonForStream: String = - """[{"topic":"test","offsetRange":[{"partition":0,"from":1,"to":100},{"partition":1,"from":1}]}]""" - val finalOffsetRangesForStreamWithoutTo: Array[OffsetRange] = getCustomOffsetRangeForReader("test".split(","), sampleJsonForStream, "STREAM") - finalOffsetRangesForStreamWithoutTo shouldEqual(sampleRangeForStream) - } - -} - diff --git a/gimel-dataapi/pom.xml b/gimel-dataapi/pom.xml index c80bac24..7592858d 100644 --- a/gimel-dataapi/pom.xml +++ b/gimel-dataapi/pom.xml @@ -45,7 +45,6 @@ under the License. gimel-connectors/gimel-hbase-1.2 gimel-connectors/gimel-cassandra-2.0 gimel-connectors/gimel-aerospike-3.14 - gimel-connectors/gimel-kafka-0.10 gimel-connectors/gimel-kafka-2.2 gimel-connectors/gimel-druid-0.82 gimel-connectors/gimel-restapi