From 34991b9df7c2ae7bc14b23416c8f4a8d4ce23a4a Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Wed, 18 Sep 2019 09:12:40 +0200 Subject: [PATCH 01/18] Initial Kafka consumer as an Exasol IMPORT UDF implementation. This is initial implementation for importing data from Kafka cluster. Currently, it only supports simple string data using String Serializers and Deserializers. --- project/Dependencies.scala | 2 + .../com/exasol/cloudetl/bucket/Bucket.scala | 2 +- .../com/exasol/cloudetl/kafka/Consumer.scala | 31 +++++++++++ .../cloudetl/scriptclasses/KafkaConsume.scala | 54 +++++++++++++++++++ .../scriptclasses/KafkaMetadata.scala | 44 +++++++++++++++ .../cloudetl/scriptclasses/KafkaPath.scala | 43 +++++++++++++++ 6 files changed, 175 insertions(+), 1 deletion(-) create mode 100644 src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala create mode 100644 src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala create mode 100644 src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala create mode 100644 src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 95b1758a..6f9e806c 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -12,6 +12,7 @@ object Dependencies { private val ParquetVersion = "1.8.1" private val AzureStorageVersion = "2.2.0" private val GoogleStorageVersion = "hadoop2-1.9.10" + private val KafkaClientsVersion = "2.3.0" private val TypesafeLoggingVersion = "3.9.0" val Resolvers: Seq[Resolver] = Seq( @@ -31,6 +32,7 @@ object Dependencies { "org.apache.parquet" % "parquet-avro" % ParquetVersion, "com.microsoft.azure" % "azure-storage" % AzureStorageVersion, "com.google.cloud.bigdataoss" % "gcs-connector" % GoogleStorageVersion, + "org.apache.kafka" % "kafka-clients" % KafkaClientsVersion, "com.typesafe.scala-logging" %% "scala-logging" % TypesafeLoggingVersion ) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala index d1a0272e..8cd4b396 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala @@ -169,7 +169,7 @@ object Bucket extends LazyLogging { * map. If it does not exist, throws an [[IllegalArgumentException]] * exception. */ - private[bucket] def requiredParam(params: Map[String, String], key: String): String = { + def requiredParam(params: Map[String, String], key: String): String = { val opt = params.get(key) opt.fold { throw new IllegalArgumentException(s"The required parameter $key is not defined!") diff --git a/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala b/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala new file mode 100644 index 00000000..db380473 --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala @@ -0,0 +1,31 @@ +package com.exasol.cloudetl.kafka + +import java.util.Properties + +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.common.serialization.StringDeserializer + +object Consumer { + + def apply(brokers: String, groupId: String): KafkaConsumer[String, String] = { + val configs = getConfiguration(brokers, groupId) + new KafkaConsumer[String, String](configs) + } + + @SuppressWarnings(Array("org.wartremover.warts.NonUnitStatements")) + private[this] def getConfiguration(brokers: String, groupId: String): Properties = { + val props = new Properties() + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) + props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + props.put( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + classOf[StringDeserializer].getCanonicalName + ) + props.put( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + classOf[StringDeserializer].getCanonicalName + ) + props + } +} diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala new file mode 100644 index 00000000..7365b47e --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala @@ -0,0 +1,54 @@ +package com.exasol.cloudetl.scriptclasses + +import java.time.Duration +import java.util.Arrays + +import scala.collection.JavaConverters._ + +import com.exasol.ExaIterator +import com.exasol.ExaMetadata +import com.exasol.cloudetl.bucket.Bucket +import com.exasol.cloudetl.kafka.Consumer + +import com.typesafe.scalalogging.LazyLogging +import org.apache.kafka.common.TopicPartition + +object KafkaConsume extends LazyLogging { + + private[this] val POLL_TIMEOUT_MS: Long = 2000L + + @SuppressWarnings(Array("org.wartremover.warts.AsInstanceOf")) + def run(meta: ExaMetadata, ctx: ExaIterator): Unit = { + val rest = ctx.getString(0) + val partitionId = ctx.getInteger(1) + val partitionOffset = ctx.getLong(2) + val partitionNextOffset = if (partitionOffset == 0) 0 else partitionOffset + 1L + val params = Bucket.keyValueStringToMap(rest) + val brokers = Bucket.requiredParam(params, "BROKER_ADDRESS") + val groupId = Bucket.requiredParam(params, "GROUP_ID") + val topics = Bucket.requiredParam(params, "TOPICS") + + val topicPartition = new TopicPartition(topics, partitionId) + val kafkaConsumer = Consumer(brokers, groupId) + kafkaConsumer.assign(Arrays.asList(topicPartition)) + kafkaConsumer.seek(topicPartition, partitionNextOffset) + + val nodeId = meta.getNodeId + val vmId = meta.getVmId + logger.info( + s"Kafka consumer for node=$nodeId, vm=$vmId using " + + s"partition=$partitionId and startOffset=$partitionNextOffset" + ) + + val records = kafkaConsumer.poll(Duration.ofMillis(POLL_TIMEOUT_MS)) + records.asScala.foreach { record => + ctx.emit( + record.partition().asInstanceOf[AnyRef], + record.offset().asInstanceOf[AnyRef], + record.value() + ) + } + kafkaConsumer.close(); + } + +} diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala new file mode 100644 index 00000000..4b49c4cd --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala @@ -0,0 +1,44 @@ +package com.exasol.cloudetl.scriptclasses + +import java.lang.{Integer => JInt} +import java.lang.{Long => JLong} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.HashMap + +import com.exasol.ExaIterator +import com.exasol.ExaMetadata +import com.exasol.cloudetl.bucket.Bucket +import com.exasol.cloudetl.kafka.Consumer + +import com.typesafe.scalalogging.LazyLogging + +object KafkaMetadata extends LazyLogging { + + @SuppressWarnings(Array("org.wartremover.warts.MutableDataStructures")) + def run(meta: ExaMetadata, iter: ExaIterator): Unit = { + val rest = iter.getString(0) + val params = Bucket.keyValueStringToMap(rest) + + val idOffsetPairs: HashMap[JInt, JLong] = HashMap.empty[JInt, JLong] + do { + val partitionId = iter.getInteger(1) + val partitionOffset = iter.getLong(2) + idOffsetPairs += (partitionId -> partitionOffset) + } while (iter.next()) + + val brokers = Bucket.requiredParam(params, "BROKER_ADDRESS") + val groupId = Bucket.requiredParam(params, "GROUP_ID") + val topics = Bucket.requiredParam(params, "TOPICS") + + val kafkaConsumer = Consumer(brokers, groupId) + val topicPartitions = kafkaConsumer.partitionsFor(topics).asScala.toList.map(_.partition()) + kafkaConsumer.close() + + topicPartitions.foreach { partitionId => + val offset: JLong = idOffsetPairs.getOrElse(partitionId, 0L) + iter.emit(new Integer(partitionId), offset) + } + } + +} diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala new file mode 100644 index 00000000..03e11e6e --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala @@ -0,0 +1,43 @@ +package com.exasol.cloudetl.scriptclasses + +import scala.collection.JavaConverters._ + +import com.exasol.ExaImportSpecification +import com.exasol.ExaMetadata +import com.exasol.cloudetl.bucket.Bucket + +object KafkaPath { + + def generateSqlForImportSpec(exaMeta: ExaMetadata, exaSpec: ExaImportSpecification): String = { + val params = exaSpec.getParameters.asScala.toMap + val tableName = Bucket.requiredParam(params, "TABLE_NAME") + val topics = Bucket.requiredParam(params, "TOPICS") + if (topics.contains(",")) { + throw new IllegalArgumentException("Only single topic can be consumed using Kafka import!") + } + val rest = Bucket.keyValueMapToString(params) + val scriptSchema = exaMeta.getScriptSchema + + s"""SELECT + | $scriptSchema.KAFKA_CONSUME( + | '$rest', partition_index, max_offset + |) + |FROM ( + | SELECT $scriptSchema.KAFKA_METADATA( + | '$rest', kafka_partition, kafka_offset + | ) FROM ( + | SELECT kafka_partition, MAX(kafka_offset) AS kafka_offset + | FROM $tableName + | GROUP BY kafka_partition + | UNION ALL + | SELECT 0, 0 + | FROM DUAL + | WHERE NOT EXISTS (SELECT * FROM $tableName) + | ) + |) + |GROUP BY + | partition_index; + |""".stripMargin + } + +} From e38f241df2ed3b9bca6fa4fde31c7bcf554417fa Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Wed, 18 Sep 2019 17:10:07 +0200 Subject: [PATCH 02/18] Use Confluent schema registry and Avro data import. Adds the support for Schema Registry and importing Avro data using Kafka Avro deserializers. Additionally, fixes a bug with initial offset value. It should start from `-1` if no data was imported before, so that next time we start consuming from zero. --- project/Dependencies.scala | 3 ++ .../cloudetl/avro/AvroRowIterator.scala | 24 +--------- .../com/exasol/cloudetl/bucket/Bucket.scala | 4 +- .../scala/com/exasol/cloudetl/data/Row.scala | 36 +++++++++++++++ .../com/exasol/cloudetl/kafka/Consumer.scala | 22 ++++++--- .../cloudetl/scriptclasses/KafkaConsume.scala | 45 ++++++++++++------- .../scriptclasses/KafkaMetadata.scala | 17 ++++--- .../cloudetl/scriptclasses/KafkaPath.scala | 2 +- 8 files changed, 98 insertions(+), 55 deletions(-) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 6f9e806c..fe99e2c8 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -13,9 +13,11 @@ object Dependencies { private val AzureStorageVersion = "2.2.0" private val GoogleStorageVersion = "hadoop2-1.9.10" private val KafkaClientsVersion = "2.3.0" + private val KafkaAvroSerializerVersion = "5.2.1" private val TypesafeLoggingVersion = "3.9.0" val Resolvers: Seq[Resolver] = Seq( + "Confluent Maven Repo" at "http://packages.confluent.io/maven/", "Exasol Releases" at "https://maven.exasol.com/artifactory/exasol-releases" ) @@ -33,6 +35,7 @@ object Dependencies { "com.microsoft.azure" % "azure-storage" % AzureStorageVersion, "com.google.cloud.bigdataoss" % "gcs-connector" % GoogleStorageVersion, "org.apache.kafka" % "kafka-clients" % KafkaClientsVersion, + "io.confluent" % "kafka-avro-serializer" % KafkaAvroSerializerVersion, "com.typesafe.scala-logging" %% "scala-logging" % TypesafeLoggingVersion ) diff --git a/src/main/scala/com/exasol/cloudetl/avro/AvroRowIterator.scala b/src/main/scala/com/exasol/cloudetl/avro/AvroRowIterator.scala index 26fcf8c8..c20bbd00 100644 --- a/src/main/scala/com/exasol/cloudetl/avro/AvroRowIterator.scala +++ b/src/main/scala/com/exasol/cloudetl/avro/AvroRowIterator.scala @@ -4,7 +4,6 @@ import com.exasol.cloudetl.data.Row import org.apache.avro.file.DataFileReader import org.apache.avro.generic.GenericRecord -import org.apache.avro.util.Utf8 /** * An object that creates a [[com.exasol.cloudetl.data.Row]] iterator @@ -37,29 +36,8 @@ object AvroRowIterator { throw new NoSuchElementException("Avro reader called next on an empty iterator!") } val record = reader.next() - recordToRow(record) + Row.fromAvroGenericRecord(record) } } - private[this] def recordToRow(record: GenericRecord): Row = { - val size = record.getSchema.getFields.size - val values = Array.ofDim[Any](size) - for { index <- 0 until size } { - values.update(index, convertRecordValue(record.get(index))) - } - Row(values.toSeq) - } - - @SuppressWarnings(Array("org.wartremover.warts.AsInstanceOf")) - private[this] def convertRecordValue(value: Any): Any = value match { - case _: GenericRecord => - throw new IllegalArgumentException("Avro nested record type is not supported yet!") - case _: java.util.Collection[_] => - throw new IllegalArgumentException("Avro collection type is not supported yet!") - case _: java.util.Map[_, _] => - throw new IllegalArgumentException("Avro map type is not supported yet!") - case _: Utf8 => value.asInstanceOf[Utf8].toString - case primitiveType => primitiveType - } - } diff --git a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala index 8cd4b396..40789b67 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala @@ -166,8 +166,8 @@ object Bucket extends LazyLogging { /** * Checks if the provided key is available in the key value parameter - * map. If it does not exist, throws an [[IllegalArgumentException]] - * exception. + * map. If it does not exist, throws an + * [[java.lang.IllegalArgumentException]] exception. */ def requiredParam(params: Map[String, String], key: String): String = { val opt = params.get(key) diff --git a/src/main/scala/com/exasol/cloudetl/data/Row.scala b/src/main/scala/com/exasol/cloudetl/data/Row.scala index 254456da..18eea3f2 100644 --- a/src/main/scala/com/exasol/cloudetl/data/Row.scala +++ b/src/main/scala/com/exasol/cloudetl/data/Row.scala @@ -1,5 +1,8 @@ package com.exasol.cloudetl.data +import org.apache.avro.generic.GenericRecord +import org.apache.avro.util.Utf8 + /** * The internal class that holds column data in an array. */ @@ -25,3 +28,36 @@ final case class Row(protected[data] val values: Seq[Any]) { values } + +/** + * A companion object to the internal [[Row]] data structure with helper + * functions. + */ +object Row { + + /** + * Returns a [[Row]] from [[org.apache.avro.generic.GenericRecord]] + * data. + */ + def fromAvroGenericRecord(record: GenericRecord): Row = { + val size = record.getSchema.getFields.size + val values = Array.ofDim[Any](size) + for { index <- 0 until size } { + values.update(index, getAvroRecordValue(record.get(index))) + } + Row(values.toSeq) + } + + @SuppressWarnings(Array("org.wartremover.warts.AsInstanceOf")) + private[this] def getAvroRecordValue(value: Any): Any = value match { + case _: GenericRecord => + throw new IllegalArgumentException("Avro nested record type is not supported yet!") + case _: java.util.Collection[_] => + throw new IllegalArgumentException("Avro collection type is not supported yet!") + case _: java.util.Map[_, _] => + throw new IllegalArgumentException("Avro map type is not supported yet!") + case _: Utf8 => value.asInstanceOf[Utf8].toString + case primitiveType => primitiveType + } + +} diff --git a/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala b/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala index db380473..497f9b22 100644 --- a/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala +++ b/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala @@ -2,29 +2,41 @@ package com.exasol.cloudetl.kafka import java.util.Properties +import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig +import io.confluent.kafka.serializers.KafkaAvroDeserializer +import org.apache.avro.generic.GenericRecord import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.common.serialization.StringDeserializer object Consumer { - def apply(brokers: String, groupId: String): KafkaConsumer[String, String] = { - val configs = getConfiguration(brokers, groupId) - new KafkaConsumer[String, String](configs) + def apply( + brokers: String, + groupId: String, + schemaRegistryUrl: String + ): KafkaConsumer[String, GenericRecord] = { + val configs = getConfiguration(brokers, groupId, schemaRegistryUrl) + new KafkaConsumer[String, GenericRecord](configs) } @SuppressWarnings(Array("org.wartremover.warts.NonUnitStatements")) - private[this] def getConfiguration(brokers: String, groupId: String): Properties = { + private[this] def getConfiguration( + brokers: String, + groupId: String, + schemaRegistryUrl: String + ): Properties = { val props = new Properties() props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + props.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl) props.put( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[StringDeserializer].getCanonicalName ) props.put( ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, - classOf[StringDeserializer].getCanonicalName + classOf[KafkaAvroDeserializer].getCanonicalName ) props } diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala index 7365b47e..ceffc8aa 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala @@ -8,6 +8,7 @@ import scala.collection.JavaConverters._ import com.exasol.ExaIterator import com.exasol.ExaMetadata import com.exasol.cloudetl.bucket.Bucket +import com.exasol.cloudetl.data.Row import com.exasol.cloudetl.kafka.Consumer import com.typesafe.scalalogging.LazyLogging @@ -22,33 +23,43 @@ object KafkaConsume extends LazyLogging { val rest = ctx.getString(0) val partitionId = ctx.getInteger(1) val partitionOffset = ctx.getLong(2) - val partitionNextOffset = if (partitionOffset == 0) 0 else partitionOffset + 1L + val partitionNextOffset = partitionOffset + 1L + val nodeId = meta.getNodeId + val vmId = meta.getVmId + logger.info( + s"Kafka consumer for node=$nodeId, vm=$vmId using " + + s"partition=$partitionId and startOffset=$partitionNextOffset" + ) + val params = Bucket.keyValueStringToMap(rest) val brokers = Bucket.requiredParam(params, "BROKER_ADDRESS") val groupId = Bucket.requiredParam(params, "GROUP_ID") + val schemaRegistryUrl = Bucket.requiredParam(params, "SCHEMA_REGISTRY_URL") val topics = Bucket.requiredParam(params, "TOPICS") val topicPartition = new TopicPartition(topics, partitionId) - val kafkaConsumer = Consumer(brokers, groupId) + val kafkaConsumer = Consumer(brokers, groupId, schemaRegistryUrl) kafkaConsumer.assign(Arrays.asList(topicPartition)) kafkaConsumer.seek(topicPartition, partitionNextOffset) - val nodeId = meta.getNodeId - val vmId = meta.getVmId - logger.info( - s"Kafka consumer for node=$nodeId, vm=$vmId using " + - s"partition=$partitionId and startOffset=$partitionNextOffset" - ) - - val records = kafkaConsumer.poll(Duration.ofMillis(POLL_TIMEOUT_MS)) - records.asScala.foreach { record => - ctx.emit( - record.partition().asInstanceOf[AnyRef], - record.offset().asInstanceOf[AnyRef], - record.value() - ) + try { + val records = kafkaConsumer.poll(Duration.ofMillis(POLL_TIMEOUT_MS)) + records.asScala.foreach { record => + logger.debug( + s"Emitting partition=${record.partition()} offset=${record.offset()} " + + s"key=${record.key()} value=${record.value()}" + ) + val metadata: Seq[Object] = + Seq(record.partition().asInstanceOf[AnyRef], record.offset().asInstanceOf[AnyRef]) + val row = Row.fromAvroGenericRecord(record.value()) + val allColumns: Seq[Object] = metadata ++ row.getValues().map(_.asInstanceOf[AnyRef]) + ctx.emit( + allColumns: _* + ) + } + } finally { + kafkaConsumer.close(); } - kafkaConsumer.close(); } } diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala index 4b49c4cd..67f79749 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala @@ -29,15 +29,18 @@ object KafkaMetadata extends LazyLogging { val brokers = Bucket.requiredParam(params, "BROKER_ADDRESS") val groupId = Bucket.requiredParam(params, "GROUP_ID") + val schemaRegistryUrl = Bucket.requiredParam(params, "SCHEMA_REGISTRY_URL") val topics = Bucket.requiredParam(params, "TOPICS") - val kafkaConsumer = Consumer(brokers, groupId) - val topicPartitions = kafkaConsumer.partitionsFor(topics).asScala.toList.map(_.partition()) - kafkaConsumer.close() - - topicPartitions.foreach { partitionId => - val offset: JLong = idOffsetPairs.getOrElse(partitionId, 0L) - iter.emit(new Integer(partitionId), offset) + val kafkaConsumer = Consumer(brokers, groupId, schemaRegistryUrl) + try { + val topicPartitions = kafkaConsumer.partitionsFor(topics).asScala.toList.map(_.partition()) + topicPartitions.foreach { partitionId => + val offset: JLong = idOffsetPairs.getOrElse(partitionId, -1) + iter.emit(new Integer(partitionId), offset) + } + } finally { + kafkaConsumer.close() } } diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala index 03e11e6e..866bcc68 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala @@ -30,7 +30,7 @@ object KafkaPath { | FROM $tableName | GROUP BY kafka_partition | UNION ALL - | SELECT 0, 0 + | SELECT 0, -1 | FROM DUAL | WHERE NOT EXISTS (SELECT * FROM $tableName) | ) From 7e8682318854a88cfd0a78a033a6dd611f968861 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Thu, 26 Sep 2019 10:59:37 +0200 Subject: [PATCH 03/18] Add optional secure (TLS) connection support to Kafka clusters. By providing the Kafka consumer keystore and truststore (JKS) files stored in Exasol BucketFS bucket, the import UDF can use them to establish secure communication with Kafka cluster. Currently, only SSL protocol is supported --- .../com/exasol/cloudetl/kafka/Consumer.scala | 43 ----- .../cloudetl/kafka/KafkaConsumerBuilder.scala | 22 +++ .../kafka/KafkaConsumerProperties.scala | 181 ++++++++++++++++++ .../{KafkaConsume.scala => KafkaImport.scala} | 13 +- .../scriptclasses/KafkaMetadata.scala | 9 +- .../cloudetl/scriptclasses/KafkaPath.scala | 4 +- 6 files changed, 214 insertions(+), 58 deletions(-) delete mode 100644 src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala create mode 100644 src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerBuilder.scala create mode 100644 src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala rename src/main/scala/com/exasol/cloudetl/scriptclasses/{KafkaConsume.scala => KafkaImport.scala} (84%) diff --git a/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala b/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala deleted file mode 100644 index 497f9b22..00000000 --- a/src/main/scala/com/exasol/cloudetl/kafka/Consumer.scala +++ /dev/null @@ -1,43 +0,0 @@ -package com.exasol.cloudetl.kafka - -import java.util.Properties - -import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig -import io.confluent.kafka.serializers.KafkaAvroDeserializer -import org.apache.avro.generic.GenericRecord -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.kafka.clients.consumer.KafkaConsumer -import org.apache.kafka.common.serialization.StringDeserializer - -object Consumer { - - def apply( - brokers: String, - groupId: String, - schemaRegistryUrl: String - ): KafkaConsumer[String, GenericRecord] = { - val configs = getConfiguration(brokers, groupId, schemaRegistryUrl) - new KafkaConsumer[String, GenericRecord](configs) - } - - @SuppressWarnings(Array("org.wartremover.warts.NonUnitStatements")) - private[this] def getConfiguration( - brokers: String, - groupId: String, - schemaRegistryUrl: String - ): Properties = { - val props = new Properties() - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) - props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) - props.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl) - props.put( - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, - classOf[StringDeserializer].getCanonicalName - ) - props.put( - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, - classOf[KafkaAvroDeserializer].getCanonicalName - ) - props - } -} diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerBuilder.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerBuilder.scala new file mode 100644 index 00000000..d8e9b901 --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerBuilder.scala @@ -0,0 +1,22 @@ +package com.exasol.cloudetl.kafka + +import org.apache.avro.generic.GenericRecord +import org.apache.kafka.clients.consumer.KafkaConsumer + +/** + * A companion object to the + * [[org.apache.kafka.clients.consumer.KafkaConsumer]] class. + */ +object KafkaConsumerBuilder { + + @SuppressWarnings(Array("org.wartremover.warts.Null")) + def apply(params: Map[String, String]): KafkaConsumer[String, GenericRecord] = { + val configs = KafkaConsumerProperties.fromImportParameters[GenericRecord](params) + new KafkaConsumer[String, GenericRecord]( + configs.getProperties(), + configs.keyDeserializerOpt.orNull, + configs.valueDeserializerOpt.orNull + ) + } + +} diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala new file mode 100644 index 00000000..137e8ee7 --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala @@ -0,0 +1,181 @@ +package com.exasol.cloudetl.kafka + +import scala.collection.JavaConverters._ +import scala.collection.mutable.{Map => MMap} + +import com.exasol.cloudetl.bucket.Bucket + +import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig +import io.confluent.kafka.serializers.KafkaAvroDeserializer +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.config.SslConfigs +import org.apache.kafka.common.serialization.Deserializer +import org.apache.kafka.common.serialization.StringDeserializer + +/** + * A companion object to the [[KafkaConsumerProperties]] class. + * + * It provides helper functions such as {@code apply} and {@code create} + * for convenient properties construction. + */ +@SuppressWarnings(Array("org.wartremover.warts.Overloading", "org.wartremover.warts.Null")) +object KafkaConsumerProperties { + + /** + * Creates Kafka consumer properties with optional key and value + * deserializers. + */ + def apply[K, V]( + properties: Map[String, String], + keyDeserializer: Option[Deserializer[K]], + valueDeserializer: Option[Deserializer[V]] + ): KafkaConsumerProperties[K, V] = { + require( + keyDeserializer != null && + (keyDeserializer.isDefined || properties + .contains(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)), + "Key deserializer should be defined or specified in properties!" + ) + require( + valueDeserializer != null && + (valueDeserializer.isDefined || properties + .contains(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)), + "Value deserializer should be defined or specified in properties!" + ) + new KafkaConsumerProperties[K, V](properties, keyDeserializer, valueDeserializer) + } + + /** + * Creates Kafka consumer properties with explicitly provided key and + * value deserializers. + */ + def apply[K, V]( + properties: Map[String, String], + keyDeserializer: Deserializer[K], + valueDeserializer: Deserializer[V] + ): KafkaConsumerProperties[K, V] = + apply(properties, Option(keyDeserializer), Option(valueDeserializer)) + + @SuppressWarnings( + Array( + "org.wartremover.warts.AsInstanceOf", + "org.wartremover.warts.MutableDataStructures", + "org.wartremover.warts.NonUnitStatements" + ) + ) + def fromImportParameters[V]( + importParams: Map[String, String] + ): KafkaConsumerProperties[String, V] = { + val params = MMap.empty[String, String] + params.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") + params.put( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, + Bucket.requiredParam(importParams, "BROKER_ADDRESS") + ) + params.put(ConsumerConfig.GROUP_ID_CONFIG, Bucket.requiredParam(importParams, "GROUP_ID")) + val schemaRegistryUrl = Bucket.requiredParam(importParams, "SCHEMA_REGISTRY_URL") + params.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl) + val sslEnabled = Bucket.optionalParameter(importParams, "SSL_ENABLED", "false") + if (sslEnabled.equals("true")) { + params.put( + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, + Bucket.requiredParam(importParams, "SECURITY_PROTOCOL") + ) + params.put( + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, + Bucket.requiredParam(importParams, "SSL_KEYSTORE_LOCATION") + ) + params.put( + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, + Bucket.requiredParam(importParams, "SSL_KEYSTORE_PASSWORD") + ) + params.put( + SslConfigs.SSL_KEY_PASSWORD_CONFIG, + Bucket.requiredParam(importParams, "SSL_KEY_PASSWORD") + ) + params.put( + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, + Bucket.requiredParam(importParams, "SSL_TRUSTSTORE_LOCATION") + ) + params.put( + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, + Bucket.requiredParam(importParams, "SSL_TRUSTSTORE_PASSWORD") + ) + } + + KafkaConsumerProperties( + params.toMap, + new StringDeserializer, + createAvroDeserializer(schemaRegistryUrl).asInstanceOf[Deserializer[V]] + ) + + } + + def createAvroDeserializer(schemaRegistryUrl: String): KafkaAvroDeserializer = { + // The schema registry url should be provided here since the one + // configured in consumer properties is not for the deserializer. + val deserializerConfig = Map( + AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG -> schemaRegistryUrl + ) + val kafkaAvroDeserializer = new KafkaAvroDeserializer + kafkaAvroDeserializer.configure(deserializerConfig.asJava, false) + kafkaAvroDeserializer + } + +} + +/** + * A properties holder class for Kafka consumers. + * + * It is parameterized on key/value deserializer types. + */ +class KafkaConsumerProperties[K, V]( + val properties: Map[String, String], + val keyDeserializerOpt: Option[Deserializer[K]], + val valueDeserializerOpt: Option[Deserializer[V]] +) { + + /** + * A comma-separated collection of host/port pairs in order to connect + * to Kafka brokers. + */ + final def withBootstrapServers(bootstrapServers: String): KafkaConsumerProperties[K, V] = + withProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers) + + /** + * A unique identifier the consumer group this consumer belongs. + */ + final def withGroupId(groupId: String): KafkaConsumerProperties[K, V] = + withProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId) + + /** + * A schema registry url this consumer can use. + */ + final def withSchemaRegistryUrl(schemaRegistryUrl: String): KafkaConsumerProperties[K, V] = + withProperty(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl) + + /** + * Sets or updates key/value Kafka consumer property. + */ + final def withProperty(key: String, value: String): KafkaConsumerProperties[K, V] = + copy(properties = properties.updated(key, value)) + + /** + * Returns the Kafka consumer properties as Java map. + */ + final def getProperties(): java.util.Map[String, AnyRef] = + properties.asInstanceOf[Map[String, AnyRef]].asJava + + @SuppressWarnings(Array("org.wartremover.warts.DefaultArguments")) + private[this] def copy( + properties: Map[String, String], + keyDeserializer: Option[Deserializer[K]] = keyDeserializerOpt, + valueDeserializer: Option[Deserializer[V]] = valueDeserializerOpt + ): KafkaConsumerProperties[K, V] = + new KafkaConsumerProperties[K, V]( + properties, + keyDeserializer, + valueDeserializer + ) +} diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala similarity index 84% rename from src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala rename to src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala index ceffc8aa..5d7704c6 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaConsume.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala @@ -9,12 +9,12 @@ import com.exasol.ExaIterator import com.exasol.ExaMetadata import com.exasol.cloudetl.bucket.Bucket import com.exasol.cloudetl.data.Row -import com.exasol.cloudetl.kafka.Consumer +import com.exasol.cloudetl.kafka.KafkaConsumerBuilder import com.typesafe.scalalogging.LazyLogging import org.apache.kafka.common.TopicPartition -object KafkaConsume extends LazyLogging { +object KafkaImport extends LazyLogging { private[this] val POLL_TIMEOUT_MS: Long = 2000L @@ -32,18 +32,16 @@ object KafkaConsume extends LazyLogging { ) val params = Bucket.keyValueStringToMap(rest) - val brokers = Bucket.requiredParam(params, "BROKER_ADDRESS") - val groupId = Bucket.requiredParam(params, "GROUP_ID") - val schemaRegistryUrl = Bucket.requiredParam(params, "SCHEMA_REGISTRY_URL") val topics = Bucket.requiredParam(params, "TOPICS") - val topicPartition = new TopicPartition(topics, partitionId) - val kafkaConsumer = Consumer(brokers, groupId, schemaRegistryUrl) + + val kafkaConsumer = KafkaConsumerBuilder(params) kafkaConsumer.assign(Arrays.asList(topicPartition)) kafkaConsumer.seek(topicPartition, partitionNextOffset) try { val records = kafkaConsumer.poll(Duration.ofMillis(POLL_TIMEOUT_MS)) + val recordsCount = records.count() records.asScala.foreach { record => logger.debug( s"Emitting partition=${record.partition()} offset=${record.offset()} " + @@ -57,6 +55,7 @@ object KafkaConsume extends LazyLogging { allColumns: _* ) } + logger.info(s"Emitted total=$recordsCount records in node=$nodeId, vm=$vmId") } finally { kafkaConsumer.close(); } diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala index 67f79749..460c3e5b 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala @@ -9,7 +9,7 @@ import scala.collection.mutable.HashMap import com.exasol.ExaIterator import com.exasol.ExaMetadata import com.exasol.cloudetl.bucket.Bucket -import com.exasol.cloudetl.kafka.Consumer +import com.exasol.cloudetl.kafka.KafkaConsumerBuilder import com.typesafe.scalalogging.LazyLogging @@ -27,14 +27,11 @@ object KafkaMetadata extends LazyLogging { idOffsetPairs += (partitionId -> partitionOffset) } while (iter.next()) - val brokers = Bucket.requiredParam(params, "BROKER_ADDRESS") - val groupId = Bucket.requiredParam(params, "GROUP_ID") - val schemaRegistryUrl = Bucket.requiredParam(params, "SCHEMA_REGISTRY_URL") + val kafkaConsumer = KafkaConsumerBuilder(params) val topics = Bucket.requiredParam(params, "TOPICS") + val topicPartitions = kafkaConsumer.partitionsFor(topics).asScala.toList.map(_.partition()) - val kafkaConsumer = Consumer(brokers, groupId, schemaRegistryUrl) try { - val topicPartitions = kafkaConsumer.partitionsFor(topics).asScala.toList.map(_.partition()) topicPartitions.foreach { partitionId => val offset: JLong = idOffsetPairs.getOrElse(partitionId, -1) iter.emit(new Integer(partitionId), offset) diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala index 866bcc68..7c1bb6cb 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala @@ -19,7 +19,7 @@ object KafkaPath { val scriptSchema = exaMeta.getScriptSchema s"""SELECT - | $scriptSchema.KAFKA_CONSUME( + | $scriptSchema.KAFKA_IMPORT( | '$rest', partition_index, max_offset |) |FROM ( @@ -32,7 +32,7 @@ object KafkaPath { | UNION ALL | SELECT 0, -1 | FROM DUAL - | WHERE NOT EXISTS (SELECT * FROM $tableName) + | WHERE NOT EXISTS (SELECT * FROM $tableName LIMIT 2) | ) |) |GROUP BY From fa3716743c6174bfaf92617bd7e060ca0e918c56 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Sun, 29 Sep 2019 12:27:09 +0200 Subject: [PATCH 04/18] Refactor Avro data types into Exasol types mapping. Previously, Avro ENUM type was considered a complex type and could not be imported into Exasol table. This commit additionally adds support for Avro Enum type import. Enum type values will be imported as string (VARCHAR) value into Exasol table. --- .../scala/com/exasol/cloudetl/data/Row.scala | 56 +++++++++++++++---- 1 file changed, 44 insertions(+), 12 deletions(-) diff --git a/src/main/scala/com/exasol/cloudetl/data/Row.scala b/src/main/scala/com/exasol/cloudetl/data/Row.scala index 18eea3f2..b755c846 100644 --- a/src/main/scala/com/exasol/cloudetl/data/Row.scala +++ b/src/main/scala/com/exasol/cloudetl/data/Row.scala @@ -1,5 +1,9 @@ package com.exasol.cloudetl.data +import scala.collection.JavaConverters._ + +import org.apache.avro.Schema +import org.apache.avro.generic.GenericFixed import org.apache.avro.generic.GenericRecord import org.apache.avro.util.Utf8 @@ -42,22 +46,50 @@ object Row { def fromAvroGenericRecord(record: GenericRecord): Row = { val size = record.getSchema.getFields.size val values = Array.ofDim[Any](size) - for { index <- 0 until size } { - values.update(index, getAvroRecordValue(record.get(index))) + record.getSchema.getFields.asScala.zipWithIndex.foreach { + case (field, index) => + values.update(index, getAvroRecordValue(record.get(index), field.schema)) } Row(values.toSeq) } - @SuppressWarnings(Array("org.wartremover.warts.AsInstanceOf")) - private[this] def getAvroRecordValue(value: Any): Any = value match { - case _: GenericRecord => - throw new IllegalArgumentException("Avro nested record type is not supported yet!") - case _: java.util.Collection[_] => - throw new IllegalArgumentException("Avro collection type is not supported yet!") - case _: java.util.Map[_, _] => - throw new IllegalArgumentException("Avro map type is not supported yet!") - case _: Utf8 => value.asInstanceOf[Utf8].toString - case primitiveType => primitiveType + @SuppressWarnings( + Array( + "org.wartremover.warts.AsInstanceOf", + "org.wartremover.warts.Null", + "org.wartremover.warts.Return", + "org.wartremover.warts.ToString" + ) + ) + def getAvroRecordValue(value: Any, field: Schema): Any = { + // scalastyle:off + if (value == null) { + return null + } + // scalastyle:on + field.getType match { + case Schema.Type.NULL => value + case Schema.Type.BOOLEAN => value + case Schema.Type.INT => value + case Schema.Type.LONG => value + case Schema.Type.FLOAT => value + case Schema.Type.DOUBLE => value + case Schema.Type.STRING => value.toString + case Schema.Type.ENUM => value.toString + case Schema.Type.UNION => getAvroUnionValue(value, field) + case Schema.Type.FIXED => value.asInstanceOf[GenericFixed].bytes().toString + case Schema.Type.BYTES => value.asInstanceOf[Utf8].toString + case field => + throw new IllegalArgumentException(s"Avro ${field.getName} type is not supperted!") + } + } + + def getAvroUnionValue(value: Any, field: Schema): Any = field.getTypes.asScala.toSeq match { + case Seq(f) => getAvroRecordValue(value, f) + case Seq(n, f) if n.getType == Schema.Type.NULL => getAvroRecordValue(value, f) + case Seq(f, n) if n.getType == Schema.Type.NULL => getAvroRecordValue(value, f) + case _ => + throw new IllegalArgumentException("Avro UNION type should contain a primitive and null!") } } From fc7d02915ef311b00e8ddb6f9ea45cd80f6afb28 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Thu, 10 Oct 2019 11:06:39 +0200 Subject: [PATCH 05/18] Apply patch from Hari (CommScope). Thanks for your feedback and contributions! Fixes #41. --- .../com/exasol/cloudetl/bucket/Bucket.scala | 14 +++++ .../kafka/KafkaConsumerProperties.scala | 17 ++++++ .../cloudetl/scriptclasses/KafkaImport.scala | 53 +++++++++++++------ .../scriptclasses/KafkaMetadata.scala | 33 ++++++++---- 4 files changed, 90 insertions(+), 27 deletions(-) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala index 40789b67..997585f0 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala @@ -124,6 +124,20 @@ object Bucket extends LazyLogging { def optionalParameter(params: Map[String, String], key: String, defaultValue: String): String = params.get(key).fold(defaultValue)(identity) + /** + * Checks whether the optional parameter is available. If it is not + * available returns the default value. + * + * @param params The parameters key value map + * @param key The optional parameter key + * @param defaultValue The default value to return if key not + * available + * @return The the value for the optional key if it exists; otherwise + * return the default value + */ + def optionalIntParameter(params: Map[String, String], key: String, defaultValue: Int): Int = + params.get(key).map(_.toInt).fold(defaultValue)(identity) + /** * Converts key value pair strings into a single string with * separators in between. diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala index 137e8ee7..a5eb4db2 100644 --- a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala @@ -76,6 +76,14 @@ object KafkaConsumerProperties { params.put(ConsumerConfig.GROUP_ID_CONFIG, Bucket.requiredParam(importParams, "GROUP_ID")) val schemaRegistryUrl = Bucket.requiredParam(importParams, "SCHEMA_REGISTRY_URL") params.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl) + params.put( + ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + Bucket.optionalParameter(importParams, "MAX_POLL_RECORDS", "500") + ) + params.put( + ConsumerConfig.FETCH_MIN_BYTES_CONFIG, + Bucket.optionalParameter(importParams, "FETCH_MIN_BYTES", "1") + ) val sslEnabled = Bucket.optionalParameter(importParams, "SSL_ENABLED", "false") if (sslEnabled.equals("true")) { params.put( @@ -102,6 +110,15 @@ object KafkaConsumerProperties { SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, Bucket.requiredParam(importParams, "SSL_TRUSTSTORE_PASSWORD") ) + val idAlgo = Bucket.optionalParameter( + importParams, + "SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", + SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM + ) + params.put( + SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, + if (idAlgo == "none") "" else idAlgo + ) } KafkaConsumerProperties( diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala index 5d7704c6..e30954de 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala @@ -16,7 +16,11 @@ import org.apache.kafka.common.TopicPartition object KafkaImport extends LazyLogging { - private[this] val POLL_TIMEOUT_MS: Long = 2000L + private[this] val POLL_TIMEOUT_MS: Int = 30000 + + private[this] val MAX_RECORDS_PER_RUN = 1000000 + + private[this] val MIN_RECORDS_PER_RUN = 100 @SuppressWarnings(Array("org.wartremover.warts.AsInstanceOf")) def run(meta: ExaMetadata, ctx: ExaIterator): Unit = { @@ -33,6 +37,11 @@ object KafkaImport extends LazyLogging { val params = Bucket.keyValueStringToMap(rest) val topics = Bucket.requiredParam(params, "TOPICS") + val timeout = Bucket.optionalIntParameter(params, "POLL_TIMEOUT_MS", POLL_TIMEOUT_MS) + val maxRecords = + Bucket.optionalIntParameter(params, "MAX_RECORDS_PER_RUN", MAX_RECORDS_PER_RUN) + val minRecords = + Bucket.optionalIntParameter(params, "MIN_RECORDS_PER_RUN", MIN_RECORDS_PER_RUN) val topicPartition = new TopicPartition(topics, partitionId) val kafkaConsumer = KafkaConsumerBuilder(params) @@ -40,22 +49,34 @@ object KafkaImport extends LazyLogging { kafkaConsumer.seek(topicPartition, partitionNextOffset) try { - val records = kafkaConsumer.poll(Duration.ofMillis(POLL_TIMEOUT_MS)) - val recordsCount = records.count() - records.asScala.foreach { record => - logger.debug( - s"Emitting partition=${record.partition()} offset=${record.offset()} " + - s"key=${record.key()} value=${record.value()}" - ) - val metadata: Seq[Object] = - Seq(record.partition().asInstanceOf[AnyRef], record.offset().asInstanceOf[AnyRef]) - val row = Row.fromAvroGenericRecord(record.value()) - val allColumns: Seq[Object] = metadata ++ row.getValues().map(_.asInstanceOf[AnyRef]) - ctx.emit( - allColumns: _* + @SuppressWarnings(Array("org.wartremover.warts.Var")) + var recordsCount = 0 + + @SuppressWarnings(Array("org.wartremover.warts.Var")) + var total = 0 + + do { + val records = kafkaConsumer.poll(Duration.ofMillis(timeout.toLong)) + recordsCount = records.count() + total += recordsCount + records.asScala.foreach { record => + logger.debug( + s"Read partition=${record.partition()} offset=${record.offset()} " + + s"key=${record.key()} value=${record.value()}" + ) + val metadata: Seq[Object] = + Seq(record.partition().asInstanceOf[AnyRef], record.offset().asInstanceOf[AnyRef]) + val row = Row.fromAvroGenericRecord(record.value()) + val allColumns: Seq[Object] = metadata ++ row.getValues().map(_.asInstanceOf[AnyRef]) + ctx.emit( + allColumns: _* + ) + } + logger.info( + s"Emitted total=$recordsCount records in node=$nodeId, vm=$vmId, partition=$partitionId" ) - } - logger.info(s"Emitted total=$recordsCount records in node=$nodeId, vm=$vmId") + + } while (recordsCount >= minRecords && total < maxRecords) } finally { kafkaConsumer.close(); } diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala index 460c3e5b..dc7dd0e7 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala @@ -5,6 +5,7 @@ import java.lang.{Long => JLong} import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap +import scala.util.{Failure, Success, Try} import com.exasol.ExaIterator import com.exasol.ExaMetadata @@ -27,18 +28,28 @@ object KafkaMetadata extends LazyLogging { idOffsetPairs += (partitionId -> partitionOffset) } while (iter.next()) - val kafkaConsumer = KafkaConsumerBuilder(params) - val topics = Bucket.requiredParam(params, "TOPICS") - val topicPartitions = kafkaConsumer.partitionsFor(topics).asScala.toList.map(_.partition()) - - try { - topicPartitions.foreach { partitionId => - val offset: JLong = idOffsetPairs.getOrElse(partitionId, -1) - iter.emit(new Integer(partitionId), offset) - } - } finally { - kafkaConsumer.close() + val kafkaConsumerTry = Try(KafkaConsumerBuilder(params)) + kafkaConsumerTry match { + + case Failure(ex) => + logger.error("Unable to create consumer", ex) + throw ex + + case Success(kafkaConsumer) => + val topics = Bucket.requiredParam(params, "TOPICS") + val topicPartitions = + kafkaConsumer.partitionsFor(topics).asScala.toList.map(_.partition()) + + try { + topicPartitions.foreach { partitionId => + val offset: JLong = idOffsetPairs.getOrElse(partitionId, -1) + iter.emit(new Integer(partitionId), offset) + } + } finally { + kafkaConsumer.close() + } } + } } From f8c660016b5331c2895f03f344fb2559f88d5538 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Fri, 4 Oct 2019 11:09:56 +0200 Subject: [PATCH 06/18] Refactor user provided parameters handling. Introduces AbstractProperties and StorageProperties classes that makes it robust and easy to maintain the UDF parameters throughout the codebase. --- .../cloudetl/bucket/AzureAdlsBucket.scala | 16 ++- .../cloudetl/bucket/AzureBlobBucket.scala | 14 +- .../com/exasol/cloudetl/bucket/Bucket.scala | 57 ++++++-- .../exasol/cloudetl/bucket/GCSBucket.scala | 14 +- .../exasol/cloudetl/bucket/LocalBucket.scala | 8 +- .../com/exasol/cloudetl/bucket/S3Bucket.scala | 16 ++- .../cloudetl/common/AbstractProperties.scala | 82 +++++++++++ .../parquet/ParquetWriteOptions.scala | 17 ++- .../exasol/cloudetl/storage/FileFormat.scala | 28 ++++ .../cloudetl/storage/StorageProperties.scala | 48 +++++++ .../exasol/cloudetl/bucket/BucketSuite.scala | 2 + .../common/AbstractPropertiesTest.scala | 127 ++++++++++++++++++ .../scriptclasses/ExportPathSuite.scala | 2 +- .../scriptclasses/ImportPathSuite.scala | 2 +- .../cloudetl/sink/BatchSizedSinkSuite.scala | 12 +- .../cloudetl/storage/FileFormatTest.scala | 27 ++++ .../storage/StoragePropertiesTest.scala | 64 +++++++++ 17 files changed, 484 insertions(+), 52 deletions(-) create mode 100644 src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala create mode 100644 src/main/scala/com/exasol/cloudetl/storage/FileFormat.scala create mode 100644 src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala create mode 100644 src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala create mode 100644 src/test/scala/com/exasol/cloudetl/storage/FileFormatTest.scala create mode 100644 src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala diff --git a/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala index a07a5d70..d371acaa 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala @@ -1,19 +1,21 @@ package com.exasol.cloudetl.bucket +import com.exasol.cloudetl.storage.StorageProperties + import org.apache.hadoop.conf.Configuration /** A [[Bucket]] implementation for the Azure Data Lake Storage */ -final case class AzureAdlsBucket(path: String, params: Map[String, String]) extends Bucket { +final case class AzureAdlsBucket(path: String, params: StorageProperties) extends Bucket { /** @inheritdoc */ override val bucketPath: String = path /** @inheritdoc */ - override val properties: Map[String, String] = params + override val properties: StorageProperties = params /** @inheritdoc */ - override def validate(): Unit = - Bucket.validate(properties, Bucket.AZURE_ADLS_PARAMETERS) + override def getRequiredProperties(): Seq[String] = + Bucket.AZURE_ADLS_PARAMETERS /** * @inheritdoc @@ -25,9 +27,9 @@ final case class AzureAdlsBucket(path: String, params: Map[String, String]) exte validate() val conf = new Configuration() - val clientId = Bucket.requiredParam(params, "AZURE_CLIENT_ID") - val clientSecret = Bucket.requiredParam(params, "AZURE_CLIENT_SECRET") - val directoryId = Bucket.requiredParam(params, "AZURE_DIRECTORY_ID") + val clientId = properties.getAs[String]("AZURE_CLIENT_ID") + val clientSecret = properties.getAs[String]("AZURE_CLIENT_SECRET") + val directoryId = properties.getAs[String]("AZURE_DIRECTORY_ID") val tokenEndpoint = s"https://login.microsoftonline.com/$directoryId/oauth2/token" conf.set("fs.adl.impl", classOf[org.apache.hadoop.fs.adl.AdlFileSystem].getName) conf.set("fs.AbstractFileSystem.adl.impl", classOf[org.apache.hadoop.fs.adl.Adl].getName) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala index 62a90987..bb47c4c0 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala @@ -1,19 +1,21 @@ package com.exasol.cloudetl.bucket +import com.exasol.cloudetl.storage.StorageProperties + import org.apache.hadoop.conf.Configuration /** A [[Bucket]] implementation for the Azure Blob Storage */ -final case class AzureBlobBucket(path: String, params: Map[String, String]) extends Bucket { +final case class AzureBlobBucket(path: String, params: StorageProperties) extends Bucket { /** @inheritdoc */ override val bucketPath: String = path /** @inheritdoc */ - override val properties: Map[String, String] = params + override val properties: StorageProperties = params /** @inheritdoc */ - override def validate(): Unit = - Bucket.validate(properties, Bucket.AZURE_BLOB_PARAMETERS) + override def getRequiredProperties(): Seq[String] = + Bucket.AZURE_BLOB_PARAMETERS /** * @inheritdoc @@ -25,8 +27,8 @@ final case class AzureBlobBucket(path: String, params: Map[String, String]) exte validate() val conf = new Configuration() - val accountName = Bucket.requiredParam(params, "AZURE_ACCOUNT_NAME") - val accountSecretKey = Bucket.requiredParam(params, "AZURE_SECRET_KEY") + val accountName = properties.getAs[String]("AZURE_ACCOUNT_NAME") + val accountSecretKey = properties.getAs[String]("AZURE_SECRET_KEY") conf.set("fs.azure", classOf[org.apache.hadoop.fs.azure.NativeAzureFileSystem].getName) conf.set("fs.wasb.impl", classOf[org.apache.hadoop.fs.azure.NativeAzureFileSystem].getName) conf.set("fs.wasbs.impl", classOf[org.apache.hadoop.fs.azure.NativeAzureFileSystem].getName) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala index 40789b67..65462512 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala @@ -4,6 +4,7 @@ import java.net.URI import scala.collection.SortedMap +import com.exasol.cloudetl.storage.StorageProperties import com.exasol.cloudetl.util.FileSystemUtil import com.typesafe.scalalogging.LazyLogging @@ -27,10 +28,41 @@ abstract class Bucket { val bucketPath: String /** The user provided key value pair properties. */ - val properties: Map[String, String] + val properties: StorageProperties + + /** + * Returns the sequence of key-value properties required for this + * specific storage class. + */ + def getRequiredProperties(): Seq[String] /** Validates that all required parameter key values are available. */ - def validate(): Unit + final def validate(): Unit = { + validateBaseProperties() + validateRequiredProperties() + } + + private[this] def validateBaseProperties(): Unit = { + if (!properties.containsKey(Bucket.BUCKET_PATH)) { + throw new IllegalArgumentException( + s"Please provide a value for the ${Bucket.BUCKET_PATH} property!" + ) + } + if (!properties.containsKey(Bucket.DATA_FORMAT)) { + throw new IllegalArgumentException( + s"Please provide a value for the ${Bucket.DATA_FORMAT} property!" + ) + } + } + + private[this] def validateRequiredProperties(): Unit = + getRequiredProperties().foreach { key => + if (!properties.containsKey(key)) { + throw new IllegalArgumentException( + s"Please provide a value for the $key property!" + ) + } + } /** * Creates a Hadoop [[org.apache.hadoop.conf.Configuration]] for this @@ -65,6 +97,9 @@ object Bucket extends LazyLogging { /** A required key string for a bucket path. */ final val BUCKET_PATH: String = "BUCKET_PATH" + /** A required key string for a data format. */ + final val DATA_FORMAT: String = "DATA_FORMAT" + /** The list of required parameter keys for AWS S3 bucket. */ final val S3_PARAMETERS: Seq[String] = Seq("S3_ENDPOINT", "S3_ACCESS_KEY", "S3_SECRET_KEY") @@ -96,15 +131,16 @@ object Bucket extends LazyLogging { * @return A [[Bucket]] class for the given path */ def apply(params: Map[String, String]): Bucket = { - val path = requiredParam(params, BUCKET_PATH) - val scheme = getScheme(path) + val properties = new StorageProperties(params) + val path = properties.getStoragePath() + val scheme = properties.getStoragePathScheme() scheme match { - case "s3a" => S3Bucket(path, params) - case "gs" => GCSBucket(path, params) - case "wasb" | "wasbs" => AzureBlobBucket(path, params) - case "adl" => AzureAdlsBucket(path, params) - case "file" => LocalBucket(path, params) + case "s3a" => S3Bucket(path, properties) + case "gs" => GCSBucket(path, properties) + case "wasb" | "wasbs" => AzureBlobBucket(path, properties) + case "adl" => AzureAdlsBucket(path, properties) + case "file" => LocalBucket(path, properties) case _ => throw new IllegalArgumentException(s"Unsupported path scheme $scheme") } @@ -176,9 +212,6 @@ object Bucket extends LazyLogging { }(identity) } - private[this] def getScheme(path: String): String = - new URI(path).getScheme - private[this] final val PARAMETER_SEPARATOR: String = ";" private[this] final val KEY_VALUE_SEPARATOR: String = ":=:" } diff --git a/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala index f341cef8..dbfb4418 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala @@ -1,19 +1,21 @@ package com.exasol.cloudetl.bucket +import com.exasol.cloudetl.storage.StorageProperties + import org.apache.hadoop.conf.Configuration /** A [[Bucket]] implementation for the Google Cloud Storage (GCS) */ -final case class GCSBucket(path: String, params: Map[String, String]) extends Bucket { +final case class GCSBucket(path: String, params: StorageProperties) extends Bucket { /** @inheritdoc */ override val bucketPath: String = path /** @inheritdoc */ - override val properties: Map[String, String] = params + override val properties: StorageProperties = params /** @inheritdoc */ - override def validate(): Unit = - Bucket.validate(properties, Bucket.GCS_PARAMETERS) + override def getRequiredProperties(): Seq[String] = + Bucket.GCS_PARAMETERS /** * @inheritdoc @@ -27,10 +29,10 @@ final case class GCSBucket(path: String, params: Map[String, String]) extends Bu val conf = new Configuration() conf.set("fs.gs.impl", classOf[com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem].getName) conf.setBoolean("fs.gs.auth.service.account.enable", true) - conf.set("fs.gs.project.id", Bucket.requiredParam(params, "GCS_PROJECT_ID")) + conf.set("fs.gs.project.id", properties.getAs[String]("GCS_PROJECT_ID")) conf.set( "fs.gs.auth.service.account.json.keyfile", - Bucket.requiredParam(params, "GCS_KEYFILE_PATH") + properties.getAs[String]("GCS_KEYFILE_PATH") ) conf diff --git a/src/main/scala/com/exasol/cloudetl/bucket/LocalBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/LocalBucket.scala index 8ff00b69..5f9e6b5f 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/LocalBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/LocalBucket.scala @@ -1,20 +1,22 @@ package com.exasol.cloudetl.bucket +import com.exasol.cloudetl.storage.StorageProperties + import org.apache.hadoop.conf.Configuration /** * A specific [[Bucket]] implementation for the local 'file:' scheme. */ -final case class LocalBucket(path: String, params: Map[String, String]) extends Bucket { +final case class LocalBucket(path: String, params: StorageProperties) extends Bucket { /** @inheritdoc */ override val bucketPath: String = path /** @inheritdoc */ - override val properties: Map[String, String] = params + override val properties: StorageProperties = params /** @inheritdoc */ - override def validate(): Unit = () + override def getRequiredProperties(): Seq[String] = Seq.empty[String] /** @inheritdoc */ override def getConfiguration(): Configuration = { diff --git a/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala index 8396fa6f..f73a3a23 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala @@ -1,19 +1,21 @@ package com.exasol.cloudetl.bucket +import com.exasol.cloudetl.storage.StorageProperties + import org.apache.hadoop.conf.Configuration /** A [[Bucket]] implementation for the AWS S3 */ -final case class S3Bucket(path: String, params: Map[String, String]) extends Bucket { +final case class S3Bucket(path: String, params: StorageProperties) extends Bucket { /** @inheritdoc */ override val bucketPath: String = path /** @inheritdoc */ - override val properties: Map[String, String] = params + override val properties: StorageProperties = params /** @inheritdoc */ - override def validate(): Unit = - Bucket.validate(properties, Bucket.S3_PARAMETERS) + override def getRequiredProperties(): Seq[String] = + Bucket.S3_PARAMETERS /** * @inheritdoc @@ -27,9 +29,9 @@ final case class S3Bucket(path: String, params: Map[String, String]) extends Buc val conf = new Configuration() conf.set("fs.file.impl", classOf[org.apache.hadoop.fs.LocalFileSystem].getName) conf.set("fs.s3a.impl", classOf[org.apache.hadoop.fs.s3a.S3AFileSystem].getName) - conf.set("fs.s3a.endpoint", Bucket.requiredParam(params, "S3_ENDPOINT")) - conf.set("fs.s3a.access.key", Bucket.requiredParam(params, "S3_ACCESS_KEY")) - conf.set("fs.s3a.secret.key", Bucket.requiredParam(params, "S3_SECRET_KEY")) + conf.set("fs.s3a.endpoint", properties.getAs[String]("S3_ENDPOINT")) + conf.set("fs.s3a.access.key", properties.getAs[String]("S3_ACCESS_KEY")) + conf.set("fs.s3a.secret.key", properties.getAs[String]("S3_SECRET_KEY")) conf } diff --git a/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala b/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala new file mode 100644 index 00000000..584b6611 --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala @@ -0,0 +1,82 @@ +package com.exasol.cloudetl.common + +/** + * An abstract class that holds the user provided key-value parameters + * when using the user-defined-functions (UDFs). + * + * This only represents the raw string key-value pairs. Specific + * implementations should extends this class to support required UDF + * key-value parameters. + */ +abstract class AbstractProperties(private val properties: Map[String, String]) { + + /** + * Checks whether the key-value properties map is empty. + */ + final def isEmpty(): Boolean = + properties.isEmpty + + /** + * Checks whether the properties contain binding to the provided key. + */ + final def containsKey(key: String): Boolean = + properties.contains(key) + + /** + * Checks whether given key value is set to {@code true}. + */ + final def isEnabled(key: String): Boolean = + containsKey(key) && "true".equalsIgnoreCase(properties.get(key).getOrElse("")) + + /** + * Checks whether the value is {@code null} for given key. + * + * Returns {@code true} if the key does not exist in the properties map. + */ + final def isNull(key: String): Boolean = + properties.get(key).fold(true) { value => + value eq null // scalastyle:ignore null + } + + /** + * Returns the optional value for the given key. + * + * @return A None if key does not exists; otherwise Some(value). + */ + final def get(key: String): Option[String] = + properties.get(key) + + /** + * Returns the count of the key-value properties. + */ + final def size(): Int = + properties.size + + @SuppressWarnings( + Array( + "org.wartremover.warts.AsInstanceOf", + "org.wartremover.warts.IsInstanceOf", + "org.wartremover.warts.Return" + ) + ) + // scalastyle:off + final override def equals(other: Any): Boolean = { + if (!other.isInstanceOf[AbstractProperties]) { + return false + } + val that = other.asInstanceOf[AbstractProperties] + if (size() != that.size()) { + return false + } + if (that.properties.equals(properties)) { + true + } else { + false + } + } + // scalastyle:on + + final override def hashCode(): Int = + properties.hashCode() + +} diff --git a/src/main/scala/com/exasol/cloudetl/parquet/ParquetWriteOptions.scala b/src/main/scala/com/exasol/cloudetl/parquet/ParquetWriteOptions.scala index 0d618605..a724367e 100644 --- a/src/main/scala/com/exasol/cloudetl/parquet/ParquetWriteOptions.scala +++ b/src/main/scala/com/exasol/cloudetl/parquet/ParquetWriteOptions.scala @@ -1,5 +1,7 @@ package com.exasol.cloudetl.parquet +import com.exasol.cloudetl.storage.StorageProperties + import org.apache.parquet.hadoop.ParquetWriter import org.apache.parquet.hadoop.metadata.CompressionCodecName @@ -14,13 +16,14 @@ final case class ParquetWriteOptions( object ParquetWriteOptions { @SuppressWarnings(Array("org.wartremover.warts.Overloading")) - def apply(params: Map[String, String]): ParquetWriteOptions = { - val compressionCodec = params.getOrElse("PARQUET_COMPRESSION_CODEC", "").toUpperCase() match { - case "SNAPPY" => CompressionCodecName.SNAPPY - case "GZIP" => CompressionCodecName.GZIP - case "LZO" => CompressionCodecName.LZO - case _ => CompressionCodecName.UNCOMPRESSED - } + def apply(params: StorageProperties): ParquetWriteOptions = { + val compressionCodec = + params.get("PARQUET_COMPRESSION_CODEC").getOrElse("").toUpperCase() match { + case "SNAPPY" => CompressionCodecName.SNAPPY + case "GZIP" => CompressionCodecName.GZIP + case "LZO" => CompressionCodecName.LZO + case _ => CompressionCodecName.UNCOMPRESSED + } val blockSize = params.get("PARQUET_BLOCK_SIZE").fold(ParquetWriter.DEFAULT_BLOCK_SIZE)(_.toInt) val pageSize = params.get("PARQUET_PAGE_SIZE").fold(ParquetWriter.DEFAULT_PAGE_SIZE)(_.toInt) diff --git a/src/main/scala/com/exasol/cloudetl/storage/FileFormat.scala b/src/main/scala/com/exasol/cloudetl/storage/FileFormat.scala new file mode 100644 index 00000000..8f79d52d --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/storage/FileFormat.scala @@ -0,0 +1,28 @@ +package com.exasol.cloudetl.storage + +/** + * A companion object for [[FileFormat]] class. + * + * It provides factory methods to create file format classes from + * strings. + */ +object FileFormat { + + def apply(fileFormat: String): FileFormat = fileFormat.toUpperCase match { + case "AVRO" => AVRO + case "FILE" => FILE + case "ORC" => ORC + case "PARQUET" => PARQUET + case _ => throw new IllegalArgumentException(s"Unsupported file format $fileFormat!") + } + +} + +/** + * An enum for supported file formats. + */ +sealed trait FileFormat extends Product with Serializable +case object AVRO extends FileFormat +case object FILE extends FileFormat +case object ORC extends FileFormat +case object PARQUET extends FileFormat diff --git a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala new file mode 100644 index 00000000..0a8636f9 --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala @@ -0,0 +1,48 @@ +package com.exasol.cloudetl.storage + +import java.net.URI + +import com.exasol.cloudetl.common.AbstractProperties + +/** + * A specific implementation of + * [[com.exasol.cloudetl.common.AbstractProperties]] that handles user + * provided key-value parameters for storage import and export + * user-defined-functions (udfs). + */ +class StorageProperties(private val properties: Map[String, String]) + extends AbstractProperties(properties) { + + import StorageProperties._ + + /** Returns the storage main path. */ + final def getStoragePath(): String = + getAs[String](BUCKET_PATH) + + /** Returns the maint storage path scheme. */ + final def getStoragePathScheme(): String = + new URI(getStoragePath()).getScheme + + /** Returns the [[FileFormat]] file format. */ + final def getFileFormat(): FileFormat = + FileFormat(getAs[String](DATA_FORMAT)) + + final def getAs[T](key: String): T = + get(key).fold { + throw new IllegalArgumentException(s"Please provide a value for the $key property!") + }(_.asInstanceOf[T]) + +} + +/** + * A companion object for [[StorageProperties]] class. + */ +object StorageProperties { + + val BUCKET_PATH: String = "BUCKET_PATH" + val DATA_FORMAT: String = "DATA_FORMAT" + + def apply(params: Map[String, String]): StorageProperties = + new StorageProperties(params) + +} diff --git a/src/test/scala/com/exasol/cloudetl/bucket/BucketSuite.scala b/src/test/scala/com/exasol/cloudetl/bucket/BucketSuite.scala index 9c1b52e7..859f954d 100644 --- a/src/test/scala/com/exasol/cloudetl/bucket/BucketSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/bucket/BucketSuite.scala @@ -62,6 +62,7 @@ class BucketSuite extends FunSuite with Matchers { test("creates an AzureBlobBucket with given parameters") { val azureBlobParams = Map( Bucket.BUCKET_PATH -> "wasbs://container@account1/parquet-bucket/", + "DATA_FORMAT" -> "AVRO", "AZURE_ACCOUNT_NAME" -> "account1", "AZURE_SECRET_KEY" -> "secret" ) @@ -87,6 +88,7 @@ class BucketSuite extends FunSuite with Matchers { test("creates an AzureAdlsBucket with provided parameters") { val params = Map( Bucket.BUCKET_PATH -> "adl://my_container.azuredatalakestore.net/orc/*", + "DATA_FORMAT" -> "CSV", "AZURE_CLIENT_ID" -> "clientX", "AZURE_CLIENT_SECRET" -> "client_secret", "AZURE_DIRECTORY_ID" -> "directory_id_secret" diff --git a/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala new file mode 100644 index 00000000..456b236f --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala @@ -0,0 +1,127 @@ +package com.exasol.cloudetl.common + +import org.scalatest.BeforeAndAfterEach +import org.scalatest.FunSuite + +@SuppressWarnings(Array("org.wartremover.warts.Var")) +class AbstractPropertiesTest extends FunSuite with BeforeAndAfterEach { + + private[this] var properties: Map[String, String] = _ + + override final def beforeEach(): Unit = { + properties = Map.empty[String, String] + () + } + + test("isEmpty returns true initially") { + assert(BaseProperties(properties).isEmpty() === true) + } + + test("containsKey returns true if key exists") { + properties = Map("key" -> "value") + assert(BaseProperties(properties).containsKey("key") === true) + } + + test("containsKey returns false if key does not exist") { + assert(BaseProperties(properties).containsKey("key") === false) + } + + test("isEnabled returns true if key is set to true") { + properties = Map("isEnabledKey" -> "TruE") + assert(BaseProperties(properties).isEnabled("isEnabledKey") === true) + } + + test("isEnabled returns false if key does not exist") { + assert(BaseProperties(properties).isEnabled("isEnabledKey") === false) + } + + test("isEnabled returns false if key is set to false") { + properties = Map("isEnabledKey" -> "false") + assert(BaseProperties(properties).isEnabled("isEnabledKey") === false) + } + + test("isEnabled returns false if key is set to null") { + properties = Map("isEnabledKey" -> null) + assert(BaseProperties(properties).isEnabled("isEnabledKey") === false) + } + + test("isNull returns true if key does not exist") { + assert(BaseProperties(properties).isNull("isNullKey") === true) + } + + test("isNull returns true if key value is set to null") { + properties = Map("isNullKey" -> null) + assert(BaseProperties(properties).isNull("isNullKey") === true) + } + + test("isNull returns false if key value is not set to null") { + properties = Map("isNullKey" -> "no") + assert(BaseProperties(properties).isNull("isNullKey") === false) + } + + test("get returns None if key does not exist") { + assert(BaseProperties(properties).get("key") === None) + } + + test("get returns Option(value) if key exists") { + properties = Map("key" -> "value") + assert(BaseProperties(properties).get("key") === Option("value")) + } + + test("size returns zero by default") { + assert(BaseProperties(properties).size() === 0) + } + + test("size returns the number of entries in properties") { + properties = Map("key1" -> "value1", "key2" -> "value2") + assert(BaseProperties(properties).size() === 2) + } + + test("equals returns true if this and other are equal") { + properties = Map("key1" -> "value1", "key2" -> "value2") + val basePropertiesThis = BaseProperties(properties) + val basePropertiesOther = BaseProperties(properties) + assert(basePropertiesThis === basePropertiesOther) + } + + test("equals returns false if this and other properties are not equal") { + properties = Map("key1" -> "value1") + val basePropertiesThis = BaseProperties(properties) + val basePropertiesOther = BaseProperties(Map("key2" -> "value2")) + assert(basePropertiesThis !== basePropertiesOther) + } + + test("equals returns false if this and other size are not equal") { + properties = Map("key1" -> "value1", "key2" -> "value2") + val basePropertiesThis = BaseProperties(properties) + val basePropertiesOther = BaseProperties(Map.empty[String, String]) + assert(basePropertiesThis !== basePropertiesOther) + } + + test("equals returns false if other is not same instance type") { + properties = Map("key1" -> "value1", "key2" -> "value2") + val basePropertiesThis = BaseProperties(properties) + val basePropertiesOther = List("a") + assert(basePropertiesThis !== basePropertiesOther) + } + + test("equals returns false if other is null") { + val basePropertiesThis = BaseProperties(properties) + assert(basePropertiesThis !== null) + } + + test("hashCode returns non-zero by default") { + val baseProperties = BaseProperties(properties) + assert(baseProperties.hashCode !== 0) + } + + test("hashCode returns the hascode of properties") { + properties = Map("a" -> "1", "b" -> "2", "c" -> "3") + val baseProperties = BaseProperties(properties) + assert(baseProperties.hashCode === properties.hashCode) + } + + private[this] case class BaseProperties(val params: Map[String, String]) + extends AbstractProperties(params) + +} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathSuite.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathSuite.scala index 6f61730c..581bdbcf 100644 --- a/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathSuite.scala @@ -49,7 +49,7 @@ class ExportPathSuite extends BaseSuite { ExportPath.generateSqlForExportSpec(exaMeta, exaSpec) } - assert(thrown.getMessage === "The required parameter S3_ACCESS_KEY is not defined!") + assert(thrown.getMessage === "Please provide a value for the S3_ACCESS_KEY property!") verify(exaSpec, times(1)).getParameters verify(exaSpec, never).getSourceColumnNames } diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathSuite.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathSuite.scala index 4c23f156..325557ff 100644 --- a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathSuite.scala @@ -48,7 +48,7 @@ class ImportPathSuite extends BaseSuite { ImportPath.generateSqlForImportSpec(exaMeta, exaSpec) } - assert(thrown.getMessage === "The required parameter S3_ACCESS_KEY is not defined!") + assert(thrown.getMessage === "Please provide a value for the S3_ACCESS_KEY property!") verify(exaSpec, times(1)).getParameters } diff --git a/src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkSuite.scala b/src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkSuite.scala index 6613b00b..7e13ec66 100644 --- a/src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkSuite.scala @@ -6,6 +6,7 @@ import com.exasol.cloudetl.TestUtils import com.exasol.cloudetl.bucket.LocalBucket import com.exasol.cloudetl.data.ExaColumnInfo import com.exasol.cloudetl.data.Row +import com.exasol.cloudetl.storage.StorageProperties import org.scalatest.BeforeAndAfterEach import org.scalatest.FunSuite @@ -14,6 +15,7 @@ import org.scalatest.FunSuite class BatchSizedSinkSuite extends FunSuite with BeforeAndAfterEach with TestUtils { private var outputPath: Path = _ + private val properties = Map("BUCKET_PATH" -> "a/path", "DATA_FORMAT" -> "avro") private val columnMetadata: Seq[ExaColumnInfo] = Seq( ExaColumnInfo("c_int", classOf[java.lang.Integer], 0, 0, 0), @@ -39,7 +41,10 @@ class BatchSizedSinkSuite extends FunSuite with BeforeAndAfterEach with TestUtil } test("export single file with default batch size") { - val bucket = LocalBucket(outputPath.toUri.toString, Map("EXPORT_BATCH_SIZE" -> "4")) + val bucket = LocalBucket( + outputPath.toUri.toString, + new StorageProperties(properties ++ Map("EXPORT_BATCH_SIZE" -> "4")) + ) val sink = new BatchSizedSink(1L, "vm1", 2, columnMetadata, bucket) rows.foreach { row => sink.write(row) @@ -49,7 +54,10 @@ class BatchSizedSinkSuite extends FunSuite with BeforeAndAfterEach with TestUtil } test("export several files with batch size smaller than total records") { - val bucket = LocalBucket(outputPath.toUri.toString, Map("EXPORT_BATCH_SIZE" -> "3")) + val bucket = LocalBucket( + outputPath.toUri.toString, + new StorageProperties(properties ++ Map("EXPORT_BATCH_SIZE" -> "3")) + ) val sink = new BatchSizedSink(1L, "vm1", 7, columnMetadata, bucket) val newRows = rows ++ rows ++ rows ++ rows.take(1) newRows.foreach { row => diff --git a/src/test/scala/com/exasol/cloudetl/storage/FileFormatTest.scala b/src/test/scala/com/exasol/cloudetl/storage/FileFormatTest.scala new file mode 100644 index 00000000..25e71e78 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/storage/FileFormatTest.scala @@ -0,0 +1,27 @@ +package com.exasol.cloudetl.storage + +import org.scalatest.FunSuite + +class FileFormatTest extends FunSuite { + + test("apply returns supported file formats") { + val testData = Map( + "AVro" -> AVRO, + "ORC" -> ORC, + "fiLE" -> FILE, + "parquet" -> PARQUET + ) + testData.foreach { + case (given, expected) => + assert(FileFormat(given) === expected) + } + } + + test("apply throws if file format is not supported") { + val thrown = intercept[IllegalArgumentException] { + FileFormat("CsV") + } + assert(thrown.getMessage === s"Unsupported file format CsV!") + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala new file mode 100644 index 00000000..991c711c --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala @@ -0,0 +1,64 @@ +package com.exasol.cloudetl.storage + +import org.scalatest.BeforeAndAfterEach +import org.scalatest.FunSuite + +@SuppressWarnings(Array("org.wartremover.warts.Var")) +class StoragePropertiesTest extends FunSuite with BeforeAndAfterEach { + + private[this] var properties: Map[String, String] = _ + + override final def beforeEach(): Unit = { + properties = Map.empty[String, String] + () + } + + test("getStoragePath returns storage path property value") { + val path = "a/bucket/path" + properties = Map(StorageProperties.BUCKET_PATH -> path) + assert(BaseProperties(properties).getStoragePath() === path) + } + + test("getStoragePath throws if storage path property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getStoragePath() + } + assert( + thrown.getMessage === s"Please provide a value for the " + + s"${StorageProperties.BUCKET_PATH} property!" + ) + } + + test("getStoragePathScheme returns path scheme value") { + val schemes = Seq("s3a", "s3", "wasbs", "adls", "file") + schemes.foreach { scheme => + val path = s"$scheme://a/path" + properties = Map(StorageProperties.BUCKET_PATH -> path) + assert(BaseProperties(properties).getStoragePathScheme() === scheme) + } + } + + test("getFileFormat returns supported file format value") { + properties = Map( + StorageProperties.BUCKET_PATH -> "path", + StorageProperties.DATA_FORMAT -> "orc" + ) + assert(BaseProperties(properties).getFileFormat() === ORC) + } + + test("getFileFormat throws if file format is not supported") { + val fileFormat = "a-non-supported-file-format" + properties = Map( + StorageProperties.BUCKET_PATH -> "path", + StorageProperties.DATA_FORMAT -> fileFormat + ) + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getFileFormat() + } + assert(thrown.getMessage === s"Unsupported file format $fileFormat!") + } + + private[this] case class BaseProperties(val params: Map[String, String]) + extends StorageProperties(params) + +} From 5565dca10aa90f386d436674d0a149967fa0abb1 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Tue, 8 Oct 2019 14:46:45 +0200 Subject: [PATCH 07/18] Implementations of Bucket class maintain their own properties. --- .../cloudetl/bucket/AzureAdlsBucket.scala | 17 ++++-- .../cloudetl/bucket/AzureBlobBucket.scala | 13 +++-- .../com/exasol/cloudetl/bucket/Bucket.scala | 58 ++++++++----------- .../exasol/cloudetl/bucket/GCSBucket.scala | 14 +++-- .../com/exasol/cloudetl/bucket/S3Bucket.scala | 14 +++-- .../cloudetl/common/AbstractProperties.scala | 17 ++++++ .../cloudetl/storage/StorageProperties.scala | 46 ++++++++++++++- .../common/AbstractPropertiesTest.scala | 14 +++++ .../storage/StoragePropertiesTest.scala | 26 +++++++++ 9 files changed, 165 insertions(+), 54 deletions(-) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala index d371acaa..5b1be6ee 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala @@ -7,15 +7,22 @@ import org.apache.hadoop.conf.Configuration /** A [[Bucket]] implementation for the Azure Data Lake Storage */ final case class AzureAdlsBucket(path: String, params: StorageProperties) extends Bucket { + private[this] val AZURE_CLIENT_ID: String = "AZURE_CLIENT_ID" + private[this] val AZURE_CLIENT_SECRET: String = "AZURE_CLIENT_SECRET" + private[this] val AZURE_DIRECTORY_ID: String = "AZURE_DIRECTORY_ID" + /** @inheritdoc */ override val bucketPath: String = path /** @inheritdoc */ override val properties: StorageProperties = params - /** @inheritdoc */ + /** + * Returns the list of required property keys for Azure Data Lake + * Storage. + */ override def getRequiredProperties(): Seq[String] = - Bucket.AZURE_ADLS_PARAMETERS + Seq(AZURE_CLIENT_ID, AZURE_CLIENT_SECRET, AZURE_DIRECTORY_ID) /** * @inheritdoc @@ -27,9 +34,9 @@ final case class AzureAdlsBucket(path: String, params: StorageProperties) extend validate() val conf = new Configuration() - val clientId = properties.getAs[String]("AZURE_CLIENT_ID") - val clientSecret = properties.getAs[String]("AZURE_CLIENT_SECRET") - val directoryId = properties.getAs[String]("AZURE_DIRECTORY_ID") + val clientId = properties.getAs[String](AZURE_CLIENT_ID) + val clientSecret = properties.getAs[String](AZURE_CLIENT_SECRET) + val directoryId = properties.getAs[String](AZURE_DIRECTORY_ID) val tokenEndpoint = s"https://login.microsoftonline.com/$directoryId/oauth2/token" conf.set("fs.adl.impl", classOf[org.apache.hadoop.fs.adl.AdlFileSystem].getName) conf.set("fs.AbstractFileSystem.adl.impl", classOf[org.apache.hadoop.fs.adl.Adl].getName) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala index bb47c4c0..a60fa872 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala @@ -7,15 +7,20 @@ import org.apache.hadoop.conf.Configuration /** A [[Bucket]] implementation for the Azure Blob Storage */ final case class AzureBlobBucket(path: String, params: StorageProperties) extends Bucket { + private[this] val AZURE_ACCOUNT_NAME: String = "AZURE_ACCOUNT_NAME" + private[this] val AZURE_SECRET_KEY: String = "AZURE_SECRET_KEY" + /** @inheritdoc */ override val bucketPath: String = path /** @inheritdoc */ override val properties: StorageProperties = params - /** @inheritdoc */ + /** + * Returns the list of required property keys for Azure Blob Storage. + */ override def getRequiredProperties(): Seq[String] = - Bucket.AZURE_BLOB_PARAMETERS + Seq(AZURE_ACCOUNT_NAME, AZURE_SECRET_KEY) /** * @inheritdoc @@ -27,8 +32,8 @@ final case class AzureBlobBucket(path: String, params: StorageProperties) extend validate() val conf = new Configuration() - val accountName = properties.getAs[String]("AZURE_ACCOUNT_NAME") - val accountSecretKey = properties.getAs[String]("AZURE_SECRET_KEY") + val accountName = properties.getAs[String](AZURE_ACCOUNT_NAME) + val accountSecretKey = properties.getAs[String](AZURE_SECRET_KEY) conf.set("fs.azure", classOf[org.apache.hadoop.fs.azure.NativeAzureFileSystem].getName) conf.set("fs.wasb.impl", classOf[org.apache.hadoop.fs.azure.NativeAzureFileSystem].getName) conf.set("fs.wasbs.impl", classOf[org.apache.hadoop.fs.azure.NativeAzureFileSystem].getName) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala index 65462512..0c68c435 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala @@ -92,6 +92,7 @@ abstract class Bucket { * Provides a factory method to create bucket and several utility * functions. */ +@SuppressWarnings(Array("org.wartremover.warts.Overloading")) object Bucket extends LazyLogging { /** A required key string for a bucket path. */ @@ -100,52 +101,39 @@ object Bucket extends LazyLogging { /** A required key string for a data format. */ final val DATA_FORMAT: String = "DATA_FORMAT" - /** The list of required parameter keys for AWS S3 bucket. */ - final val S3_PARAMETERS: Seq[String] = - Seq("S3_ENDPOINT", "S3_ACCESS_KEY", "S3_SECRET_KEY") - /** - * The list of required parameter keys for Google Cloud Storage - * bucket. - */ - final val GCS_PARAMETERS: Seq[String] = - Seq("GCS_PROJECT_ID", "GCS_KEYFILE_PATH") - - /** - * The list of required parameter keys for Azure Blob Storage bucket. - */ - final val AZURE_BLOB_PARAMETERS: Seq[String] = - Seq("AZURE_ACCOUNT_NAME", "AZURE_SECRET_KEY") - - /** - * The list of required keys for Azure Data Lake Storage bucket. - */ - final val AZURE_ADLS_PARAMETERS: Seq[String] = - Seq("AZURE_CLIENT_ID", "AZURE_CLIENT_SECRET", "AZURE_DIRECTORY_ID") - - /** - * An apply method that creates different [[Bucket]] classes depending - * on the path scheme. + * Creates specific [[Bucket]] class using the path scheme from + * [[com.exasol.cloudetl.storage.StorageProperties]] properties. * - * @param params The key value parameters + * @param storageProperties The user provided storage key-value + * properties * @return A [[Bucket]] class for the given path */ - def apply(params: Map[String, String]): Bucket = { - val properties = new StorageProperties(params) - val path = properties.getStoragePath() - val scheme = properties.getStoragePathScheme() + def apply(storageProperties: StorageProperties): Bucket = { + val path = storageProperties.getStoragePath() + val scheme = storageProperties.getStoragePathScheme() scheme match { - case "s3a" => S3Bucket(path, properties) - case "gs" => GCSBucket(path, properties) - case "wasb" | "wasbs" => AzureBlobBucket(path, properties) - case "adl" => AzureAdlsBucket(path, properties) - case "file" => LocalBucket(path, properties) + case "s3a" => S3Bucket(path, storageProperties) + case "gs" => GCSBucket(path, storageProperties) + case "wasb" | "wasbs" => AzureBlobBucket(path, storageProperties) + case "adl" => AzureAdlsBucket(path, storageProperties) + case "file" => LocalBucket(path, storageProperties) case _ => throw new IllegalArgumentException(s"Unsupported path scheme $scheme") } } + /** + * Creates specific [[Bucket]] class using the path scheme from + * key-value properties. + * + * @param params The key value parameters + * @return A [[Bucket]] class for the given path + */ + def apply(params: Map[String, String]): Bucket = + apply(StorageProperties(params)) + /** * Checks whether the optional parameter is available. If it is not * available returns the default value. diff --git a/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala index dbfb4418..f51dbc92 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala @@ -7,15 +7,21 @@ import org.apache.hadoop.conf.Configuration /** A [[Bucket]] implementation for the Google Cloud Storage (GCS) */ final case class GCSBucket(path: String, params: StorageProperties) extends Bucket { + private[this] val GCS_PROJECT_ID: String = "GCS_PROJECT_ID" + private[this] val GCS_KEYFILE_PATH: String = "GCS_KEYFILE_PATH" + /** @inheritdoc */ override val bucketPath: String = path /** @inheritdoc */ override val properties: StorageProperties = params - /** @inheritdoc */ + /** + * Returns the list of required property keys for Google Cloud + * Storage. + */ override def getRequiredProperties(): Seq[String] = - Bucket.GCS_PARAMETERS + Seq(GCS_PROJECT_ID, GCS_KEYFILE_PATH) /** * @inheritdoc @@ -29,10 +35,10 @@ final case class GCSBucket(path: String, params: StorageProperties) extends Buck val conf = new Configuration() conf.set("fs.gs.impl", classOf[com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem].getName) conf.setBoolean("fs.gs.auth.service.account.enable", true) - conf.set("fs.gs.project.id", properties.getAs[String]("GCS_PROJECT_ID")) + conf.set("fs.gs.project.id", properties.getAs[String](GCS_PROJECT_ID)) conf.set( "fs.gs.auth.service.account.json.keyfile", - properties.getAs[String]("GCS_KEYFILE_PATH") + properties.getAs[String](GCS_KEYFILE_PATH) ) conf diff --git a/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala index f73a3a23..2abe8321 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala @@ -7,15 +7,19 @@ import org.apache.hadoop.conf.Configuration /** A [[Bucket]] implementation for the AWS S3 */ final case class S3Bucket(path: String, params: StorageProperties) extends Bucket { + private[this] val S3_ENDPOINT: String = "S3_ENDPOINT" + private[this] val S3_ACCESS_KEY: String = "S3_ACCESS_KEY" + private[this] val S3_SECRET_KEY: String = "S3_SECRET_KEY" + /** @inheritdoc */ override val bucketPath: String = path /** @inheritdoc */ override val properties: StorageProperties = params - /** @inheritdoc */ + /** Returns the list of required property keys for AWS S3 Storage. */ override def getRequiredProperties(): Seq[String] = - Bucket.S3_PARAMETERS + Seq(S3_ENDPOINT, S3_ACCESS_KEY, S3_SECRET_KEY) /** * @inheritdoc @@ -29,9 +33,9 @@ final case class S3Bucket(path: String, params: StorageProperties) extends Bucke val conf = new Configuration() conf.set("fs.file.impl", classOf[org.apache.hadoop.fs.LocalFileSystem].getName) conf.set("fs.s3a.impl", classOf[org.apache.hadoop.fs.s3a.S3AFileSystem].getName) - conf.set("fs.s3a.endpoint", properties.getAs[String]("S3_ENDPOINT")) - conf.set("fs.s3a.access.key", properties.getAs[String]("S3_ACCESS_KEY")) - conf.set("fs.s3a.secret.key", properties.getAs[String]("S3_SECRET_KEY")) + conf.set("fs.s3a.endpoint", properties.getAs[String](S3_ENDPOINT)) + conf.set("fs.s3a.access.key", properties.getAs[String](S3_ACCESS_KEY)) + conf.set("fs.s3a.secret.key", properties.getAs[String](S3_SECRET_KEY)) conf } diff --git a/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala b/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala index 584b6611..7fff3645 100644 --- a/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala @@ -1,5 +1,7 @@ package com.exasol.cloudetl.common +import scala.collection.SortedMap + /** * An abstract class that holds the user provided key-value parameters * when using the user-defined-functions (UDFs). @@ -52,6 +54,21 @@ abstract class AbstractProperties(private val properties: Map[String, String]) { final def size(): Int = properties.size + /** + * Returns a string listing of all key-value property pairs. + * + * The resulting string contains key-value pairs in a sorted order by + * keys. + * + * @param keyValueSeparator The separator between each key-value pairs + * @param propertySeparator The separator between each key-value pair strings + * @return The string value of properties with provided separators + */ + final def mkString(keyValueSeparator: String, propertySeparator: String): String = + (SortedMap.empty[String, String] ++ properties) + .map { case (k, v) => s"$k$keyValueSeparator$v" } + .mkString(propertySeparator) + @SuppressWarnings( Array( "org.wartremover.warts.AsInstanceOf", diff --git a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala index 0a8636f9..79adf55b 100644 --- a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala @@ -19,7 +19,7 @@ class StorageProperties(private val properties: Map[String, String]) final def getStoragePath(): String = getAs[String](BUCKET_PATH) - /** Returns the maint storage path scheme. */ + /** Returns the main storage path scheme. */ final def getStoragePathScheme(): String = new URI(getStoragePath()).getScheme @@ -27,11 +27,23 @@ class StorageProperties(private val properties: Map[String, String]) final def getFileFormat(): FileFormat = FileFormat(getAs[String](DATA_FORMAT)) + final def getParallelism(defaultValue: => String): String = + get(PARALLELISM).fold(defaultValue)(identity) + final def getAs[T](key: String): T = get(key).fold { throw new IllegalArgumentException(s"Please provide a value for the $key property!") }(_.asInstanceOf[T]) + /** + * Returns a string value of key-value property pairs. + * + * The resulting string is sorted by keys ordering. + */ + @SuppressWarnings(Array("org.wartremover.warts.Overloading")) + final def mkString(): String = + mkString(KEY_VALUE_SEPARATOR, PROPERTY_SEPARATOR) + } /** @@ -39,10 +51,42 @@ class StorageProperties(private val properties: Map[String, String]) */ object StorageProperties { + /** + * A line separator string used for creating key-value property + * strings. + */ + private[storage] final val PROPERTY_SEPARATOR: String = ";" + + /** + * A default separator string used for concatenate key-value pairs. + */ + private[storage] final val KEY_VALUE_SEPARATOR: String = " -> " + val BUCKET_PATH: String = "BUCKET_PATH" val DATA_FORMAT: String = "DATA_FORMAT" + val PARALLELISM: String = "PARALLELISM" def apply(params: Map[String, String]): StorageProperties = new StorageProperties(params) + /** + * Creates [[StorageProperties]] from properly separated string. + */ + def fromString(string: String): StorageProperties = { + if (!string.contains(PROPERTY_SEPARATOR)) { + throw new IllegalArgumentException( + s"The input string is not separated by '$PROPERTY_SEPARATOR'!" + ) + } + val properties = string + .split(PROPERTY_SEPARATOR) + .map { word => + val pairs = word.split(KEY_VALUE_SEPARATOR) + pairs(0) -> pairs(1) + } + .toMap + + new StorageProperties(properties) + } + } diff --git a/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala index 456b236f..15ba6496 100644 --- a/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala +++ b/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala @@ -77,6 +77,20 @@ class AbstractPropertiesTest extends FunSuite with BeforeAndAfterEach { assert(BaseProperties(properties).size() === 2) } + test("mkString returns empty string by default") { + val str = BaseProperties(properties).mkString("->", "=") + assert(str.isEmpty === true) + assert(str === "") + } + + test("mkString returns separated key-value property pairs") { + properties = Map("k1" -> "v1", "k2" -> "v2", "a" -> "1") + val expected = s"a -> 1;k1 -> v1;k2 -> v2" // sorted + val str = BaseProperties(properties).mkString(" -> ", ";") + assert(str.isEmpty === false) + assert(str === expected) + } + test("equals returns true if this and other are equal") { properties = Map("key1" -> "value1", "key2" -> "value2") val basePropertiesThis = BaseProperties(properties) diff --git a/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala index 991c711c..f14c72e8 100644 --- a/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala +++ b/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala @@ -58,6 +58,32 @@ class StoragePropertiesTest extends FunSuite with BeforeAndAfterEach { assert(thrown.getMessage === s"Unsupported file format $fileFormat!") } + test("mkString returns empty string by default") { + val str = BaseProperties(properties).mkString() + assert(str.isEmpty === true) + assert(str === "") + } + + test("mkString returns key-value properties string") { + properties = Map("k1" -> "v1", "k3" -> "v3", "k2" -> "v2") + val expected = "k1 -> v1;k2 -> v2;k3 -> v3" + assert(BaseProperties(properties).mkString() === expected) + } + + test("fromString throws if input string does not contain separator") { + val thrown = intercept[IllegalArgumentException] { + StorageProperties.fromString("") + } + assert(thrown.getMessage === s"The input string is not separated by ';'!") + } + + test("fromString returns correct StorageProperties") { + properties = Map("k3" -> "v3", "k2" -> "v2") + val baseProperty = BaseProperties(properties) + val mkStringResult = baseProperty.mkString() + assert(StorageProperties.fromString(mkStringResult) === baseProperty) + } + private[this] case class BaseProperties(val params: Map[String, String]) extends StorageProperties(params) From 6508d2ee47ecec30a2d50921fafef78b5ee57e8b Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Wed, 9 Oct 2019 23:29:59 +0200 Subject: [PATCH 08/18] Fix and update tests. --- project/Compilation.scala | 3 +- .../com/exasol/cloudetl/bucket/Bucket.scala | 25 +---- .../cloudetl/scriptclasses/ExportPath.scala | 26 +++--- .../cloudetl/scriptclasses/ExportTable.scala | 22 ++--- .../cloudetl/scriptclasses/ImportFiles.scala | 39 +++----- .../scriptclasses/ImportMetadata.scala | 21 ++--- .../cloudetl/scriptclasses/ImportPath.scala | 26 +++--- .../com/exasol/cloudetl/source/Source.scala | 25 +++++ .../exasol/cloudetl/storage/FileFormat.scala | 9 +- .../cloudetl/storage/StorageProperties.scala | 11 ++- ...TestUtils.scala => DummyRecordsTest.scala} | 2 +- .../{BucketSuite.scala => BucketTest.scala} | 69 +++++++------- .../common/AbstractPropertiesTest.scala | 1 - ...rSuite.scala => OrcDeserializerTest.scala} | 11 +-- .../cloudetl/scriptclasses/BaseSuite.scala | 51 ---------- .../scriptclasses/ExportPathSuite.scala | 77 --------------- .../scriptclasses/ExportPathTest.scala | 66 +++++++++++++ ...TableSuite.scala => ExportTableTest.scala} | 59 ++++++------ .../scriptclasses/ImportFilesSuite.scala | 93 ------------------- .../scriptclasses/ImportFilesTest.scala | 92 ++++++++++++++++++ .../scriptclasses/ImportMetadataSuite.scala | 25 ----- .../scriptclasses/ImportMetadataTest.scala | 32 +++++++ .../scriptclasses/ImportPathSuite.scala | 55 ----------- .../scriptclasses/ImportPathTest.scala | 50 ++++++++++ .../cloudetl/scriptclasses/PathTest.scala | 37 ++++++++ .../cloudetl/scriptclasses/StorageTest.scala | 38 ++++++++ ...nkSuite.scala => BatchSizedSinkTest.scala} | 5 +- .../cloudetl/source/AvroSourceSuite.scala | 41 -------- .../cloudetl/source/AvroSourceTest.scala | 14 +++ .../cloudetl/source/OrcSourceSuite.scala | 54 ----------- .../cloudetl/source/OrcSourceTest.scala | 19 ++++ .../cloudetl/source/ParquetSourceSuite.scala | 86 ----------------- .../cloudetl/source/ParquetSourceTest.scala | 50 ++++++++++ .../exasol/cloudetl/source/SourceTest.scala | 43 +++++++++ .../cloudetl/storage/FileFormatTest.scala | 2 + .../storage/StoragePropertiesTest.scala | 3 +- ...UtilSuite.scala => DateTimeUtilTest.scala} | 2 +- ...ilSuite.scala => FileSystemUtilTest.scala} | 20 ++-- ...maUtilSuite.scala => SchemaUtilTest.scala} | 18 ++-- 39 files changed, 642 insertions(+), 680 deletions(-) rename src/test/scala/com/exasol/cloudetl/{TestUtils.scala => DummyRecordsTest.scala} (98%) rename src/test/scala/com/exasol/cloudetl/bucket/{BucketSuite.scala => BucketTest.scala} (67%) rename src/test/scala/com/exasol/cloudetl/orc/{OrcDeserializerSuite.scala => OrcDeserializerTest.scala} (73%) delete mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/BaseSuite.scala delete mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathSuite.scala create mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathTest.scala rename src/test/scala/com/exasol/cloudetl/scriptclasses/{ExportTableSuite.scala => ExportTableTest.scala} (63%) delete mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/ImportFilesSuite.scala create mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/ImportFilesTest.scala delete mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/ImportMetadataSuite.scala create mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/ImportMetadataTest.scala delete mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathSuite.scala create mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathTest.scala create mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/PathTest.scala create mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/StorageTest.scala rename src/test/scala/com/exasol/cloudetl/sink/{BatchSizedSinkSuite.scala => BatchSizedSinkTest.scala} (92%) delete mode 100644 src/test/scala/com/exasol/cloudetl/source/AvroSourceSuite.scala create mode 100644 src/test/scala/com/exasol/cloudetl/source/AvroSourceTest.scala delete mode 100644 src/test/scala/com/exasol/cloudetl/source/OrcSourceSuite.scala create mode 100644 src/test/scala/com/exasol/cloudetl/source/OrcSourceTest.scala delete mode 100644 src/test/scala/com/exasol/cloudetl/source/ParquetSourceSuite.scala create mode 100644 src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala create mode 100644 src/test/scala/com/exasol/cloudetl/source/SourceTest.scala rename src/test/scala/com/exasol/cloudetl/util/{DateTimeUtilSuite.scala => DateTimeUtilTest.scala} (97%) rename src/test/scala/com/exasol/cloudetl/util/{FileSystemUtilSuite.scala => FileSystemUtilTest.scala} (68%) rename src/test/scala/com/exasol/cloudetl/util/{SchemaUtilSuite.scala => SchemaUtilTest.scala} (92%) diff --git a/project/Compilation.scala b/project/Compilation.scala index 4de11ec9..1d2732c8 100644 --- a/project/Compilation.scala +++ b/project/Compilation.scala @@ -126,7 +126,8 @@ object Compilation { val WartremoverTestFlags: Seq[Wart] = ExtraWartremoverFlags ++ Warts.allBut( Wart.Any, Wart.NonUnitStatements, - Wart.Null + Wart.Null, + Wart.Var ) } diff --git a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala index 0c68c435..6df52e47 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala @@ -37,23 +37,8 @@ abstract class Bucket { def getRequiredProperties(): Seq[String] /** Validates that all required parameter key values are available. */ - final def validate(): Unit = { - validateBaseProperties() + final def validate(): Unit = validateRequiredProperties() - } - - private[this] def validateBaseProperties(): Unit = { - if (!properties.containsKey(Bucket.BUCKET_PATH)) { - throw new IllegalArgumentException( - s"Please provide a value for the ${Bucket.BUCKET_PATH} property!" - ) - } - if (!properties.containsKey(Bucket.DATA_FORMAT)) { - throw new IllegalArgumentException( - s"Please provide a value for the ${Bucket.DATA_FORMAT} property!" - ) - } - } private[this] def validateRequiredProperties(): Unit = getRequiredProperties().foreach { key => @@ -95,12 +80,6 @@ abstract class Bucket { @SuppressWarnings(Array("org.wartremover.warts.Overloading")) object Bucket extends LazyLogging { - /** A required key string for a bucket path. */ - final val BUCKET_PATH: String = "BUCKET_PATH" - - /** A required key string for a data format. */ - final val DATA_FORMAT: String = "DATA_FORMAT" - /** * Creates specific [[Bucket]] class using the path scheme from * [[com.exasol.cloudetl.storage.StorageProperties]] properties. @@ -120,7 +99,7 @@ object Bucket extends LazyLogging { case "adl" => AzureAdlsBucket(path, storageProperties) case "file" => LocalBucket(path, storageProperties) case _ => - throw new IllegalArgumentException(s"Unsupported path scheme $scheme") + throw new IllegalArgumentException(s"Unsupported path scheme $scheme!") } } diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportPath.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportPath.scala index 5f12a3ca..dfb686b0 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportPath.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportPath.scala @@ -4,28 +4,30 @@ import scala.collection.JavaConverters._ import com.exasol.ExaExportSpecification import com.exasol.ExaMetadata -import com.exasol.cloudetl.bucket._ +import com.exasol.cloudetl.bucket.Bucket +import com.exasol.cloudetl.storage.StorageProperties object ExportPath { - def generateSqlForExportSpec(exaMeta: ExaMetadata, exaSpec: ExaExportSpecification): String = { - val params = exaSpec.getParameters.asScala.toMap - val bucket = Bucket(params) - + def generateSqlForExportSpec( + metadata: ExaMetadata, + exportSpec: ExaExportSpecification + ): String = { + val storageProperties = StorageProperties(exportSpec.getParameters.asScala.toMap) + val bucket = Bucket(storageProperties) bucket.validate() val bucketPath = bucket.bucketPath - val parallelism = Bucket.optionalParameter(params, "PARALLELISM", "iproc()") - val rest = Bucket.keyValueMapToString(params) - - val scriptSchema = exaMeta.getScriptSchema + val parallelism = storageProperties.getParallelism("iproc()") + val storagePropertiesStr = storageProperties.mkString() + val scriptSchema = metadata.getScriptSchema - val srcColumns = getSourceColumns(exaSpec) - val srcColumnsParam = srcColumns.mkString(".") + val srcColumns = getSourceColumns(exportSpec) + val srcColumnsStr = srcColumns.mkString(".") s"""SELECT | $scriptSchema.EXPORT_TABLE( - | '$bucketPath', '$rest', '$srcColumnsParam', ${srcColumns.mkString(", ")} + | '$bucketPath', '$storagePropertiesStr', '$srcColumnsStr', ${srcColumns.mkString(", ")} |) |FROM | DUAL diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportTable.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportTable.scala index 800258c2..3e1db24b 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportTable.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportTable.scala @@ -8,6 +8,7 @@ import com.exasol.cloudetl.bucket.Bucket import com.exasol.cloudetl.data.ExaColumnInfo import com.exasol.cloudetl.data.Row import com.exasol.cloudetl.sink.BatchSizedSink +import com.exasol.cloudetl.storage.StorageProperties import com.exasol.cloudetl.util.SchemaUtil import com.typesafe.scalalogging.LazyLogging @@ -15,25 +16,24 @@ import com.typesafe.scalalogging.LazyLogging @SuppressWarnings(Array("org.wartremover.warts.Var")) object ExportTable extends LazyLogging { - def run(meta: ExaMetadata, iter: ExaIterator): Unit = { - val params = Bucket.keyValueStringToMap(iter.getString(1)) - val bucket = Bucket(params) - - val srcColumnNames = iter.getString(2).split("\\.") + def run(metadata: ExaMetadata, iterator: ExaIterator): Unit = { + val storageProperties = StorageProperties.fromString(iterator.getString(1)) + val bucket = Bucket(storageProperties) + val srcColumnNames = iterator.getString(2).split("\\.") val firstColumnIdx = 3 - val nodeId = meta.getNodeId - val vmId = meta.getVmId - val columns = getColumns(meta, srcColumnNames, firstColumnIdx) + val nodeId = metadata.getNodeId + val vmId = metadata.getVmId + val columns = getColumns(metadata, srcColumnNames, firstColumnIdx) - val sink = new BatchSizedSink(nodeId, vmId, iter.size(), columns, bucket) + val sink = new BatchSizedSink(nodeId, vmId, iterator.size(), columns, bucket) logger.info(s"Starting export from node: $nodeId, vm: $vmId.") do { - val row = getRow(iter, firstColumnIdx, columns) + val row = getRow(iterator, firstColumnIdx, columns) sink.write(row) - } while (iter.next()) + } while (iterator.next()) sink.close() diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportFiles.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportFiles.scala index dae280e0..34952e1d 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportFiles.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportFiles.scala @@ -4,56 +4,47 @@ import scala.collection.mutable.ListBuffer import com.exasol.ExaIterator import com.exasol.ExaMetadata -import com.exasol.cloudetl.bucket._ +import com.exasol.cloudetl.bucket.Bucket import com.exasol.cloudetl.data.Row import com.exasol.cloudetl.source._ +import com.exasol.cloudetl.storage.StorageProperties import com.typesafe.scalalogging.LazyLogging import org.apache.hadoop.fs.Path object ImportFiles extends LazyLogging { - def run(meta: ExaMetadata, ctx: ExaIterator): Unit = { - val rest = ctx.getString(1) - val params = Bucket.keyValueStringToMap(rest) - val format = Bucket.optionalParameter(params, "DATA_FORMAT", "PARQUET") - val bucket = Bucket(params) + def run(metadata: ExaMetadata, iterator: ExaIterator): Unit = { + val storageProperties = StorageProperties.fromString(iterator.getString(1)) + val fileFormat = storageProperties.getFileFormat() + val bucket = Bucket(storageProperties) - val files = groupFiles(ctx, 2) - val nodeId = meta.getNodeId - val vmId = meta.getVmId + val files = groupFiles(iterator, 2) + val nodeId = metadata.getNodeId + val vmId = metadata.getVmId logger.info(s"The total number of files for node: $nodeId, vm: $vmId is '${files.size}'.") files.foreach { file => logger.debug(s"Importing from file: '$file'") - val source = createSource(format, file, bucket) - readAndEmit(source.stream(), ctx) + val source = + Source(fileFormat, new Path(file), bucket.getConfiguration(), bucket.fileSystem) + readAndEmit(source.stream(), iterator) source.close() } } @SuppressWarnings(Array("org.wartremover.warts.MutableDataStructures")) private[this] def groupFiles( - exasolIterator: ExaIterator, + iterator: ExaIterator, fileStartingIndex: Int ): Seq[String] = { val files = ListBuffer[String]() do { - files.append(exasolIterator.getString(fileStartingIndex)) - } while (exasolIterator.next()) + files.append(iterator.getString(fileStartingIndex)) + } while (iterator.next()) files.toSeq } - private[this] def createSource(format: String, file: String, bucket: Bucket): Source = - format.toLowerCase match { - case "avro" => AvroSource(new Path(file), bucket.getConfiguration(), bucket.fileSystem) - case "orc" => OrcSource(new Path(file), bucket.getConfiguration(), bucket.fileSystem) - case "parquet" => - ParquetSource(new Path(file), bucket.getConfiguration(), bucket.fileSystem) - case _ => - throw new IllegalArgumentException(s"Unsupported storage format: '$format'") - } - private[this] def readAndEmit(rowIterator: Iterator[Row], ctx: ExaIterator): Unit = rowIterator.foreach { row => val columns: Seq[Object] = row.getValues().map(_.asInstanceOf[AnyRef]) diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportMetadata.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportMetadata.scala index 47ca17b6..852ab938 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportMetadata.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportMetadata.scala @@ -2,30 +2,29 @@ package com.exasol.cloudetl.scriptclasses import com.exasol.ExaIterator import com.exasol.ExaMetadata -import com.exasol.cloudetl.bucket._ +import com.exasol.cloudetl.bucket.Bucket +import com.exasol.cloudetl.storage.StorageProperties import com.typesafe.scalalogging.LazyLogging object ImportMetadata extends LazyLogging { - def run(meta: ExaMetadata, iter: ExaIterator): Unit = { - val bucketPath = iter.getString(0) - val parallelism = iter.getInteger(2) - + def run(metadata: ExaMetadata, iterator: ExaIterator): Unit = { + val bucketPath = iterator.getString(0) + val parallelism = iterator.getInteger(2) logger.info( - s"Reading metadata from bucket path: $bucketPath with parallelism: ${parallelism.toString}" + s"Reading metadata from bucket path: $bucketPath " + + s"with parallelism: ${parallelism.toString}" ) - val rest = iter.getString(1) - val params = Bucket.keyValueStringToMap(rest) - val bucket = Bucket(params) - + val storageProperties = StorageProperties.fromString(iterator.getString(1)) + val bucket = Bucket(storageProperties) val paths = bucket.getPaths().filter(p => !p.getName().startsWith("_")) logger.info(s"Total number of files: ${paths.size} in bucket path: $bucketPath") paths.zipWithIndex.foreach { case (filename, idx) => - iter.emit(filename.toString, s"${idx % parallelism}") + iterator.emit(filename.toString, s"${idx % parallelism}") } } diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportPath.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportPath.scala index 8d59fd9d..470026ad 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportPath.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportPath.scala @@ -4,31 +4,31 @@ import scala.collection.JavaConverters._ import com.exasol.ExaImportSpecification import com.exasol.ExaMetadata -import com.exasol.cloudetl.bucket._ +import com.exasol.cloudetl.bucket.Bucket +import com.exasol.cloudetl.storage.StorageProperties object ImportPath { - def generateSqlForImportSpec(exaMeta: ExaMetadata, exaSpec: ExaImportSpecification): String = { - val params = exaSpec.getParameters.asScala.toMap - - val bucket = Bucket(params) - + def generateSqlForImportSpec( + metadata: ExaMetadata, + importSpec: ExaImportSpecification + ): String = { + val storageProperties = StorageProperties(importSpec.getParameters.asScala.toMap) + val bucket = Bucket(storageProperties) bucket.validate() val bucketPath = bucket.bucketPath - val parallelism = Bucket.optionalParameter(params, "PARALLELISM", "nproc()") - - val rest = Bucket.keyValueMapToString(params) - - val scriptSchema = exaMeta.getScriptSchema + val parallelism = storageProperties.getParallelism("nproc()") + val storagePropertiesStr = storageProperties.mkString() + val scriptSchema = metadata.getScriptSchema s"""SELECT | $scriptSchema.IMPORT_FILES( - | '$bucketPath', '$rest', filename + | '$bucketPath', '$storagePropertiesStr', filename |) |FROM ( | SELECT $scriptSchema.IMPORT_METADATA( - | '$bucketPath', '$rest', $parallelism + | '$bucketPath', '$storagePropertiesStr', $parallelism | ) |) |GROUP BY diff --git a/src/main/scala/com/exasol/cloudetl/source/Source.scala b/src/main/scala/com/exasol/cloudetl/source/Source.scala index 88d176ff..6b3fa803 100644 --- a/src/main/scala/com/exasol/cloudetl/source/Source.scala +++ b/src/main/scala/com/exasol/cloudetl/source/Source.scala @@ -1,6 +1,8 @@ package com.exasol.cloudetl.source import com.exasol.cloudetl.data.Row +import com.exasol.cloudetl.storage.FileFormat +import com.exasol.cloudetl.storage.FileFormat._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem @@ -37,3 +39,26 @@ abstract class Source { def close(): Unit } + +/** + * A companion object to the [[Source]] class. + * + * Provides a helper methods to create specific implementations of + * Source class. + */ +object Source { + + def apply( + fileFormat: FileFormat, + filePath: Path, + conf: Configuration, + fileSystem: FileSystem + ): Source = fileFormat match { + case AVRO => AvroSource(filePath, conf, fileSystem) + case ORC => OrcSource(filePath, conf, fileSystem) + case PARQUET => ParquetSource(filePath, conf, fileSystem) + case _ => + throw new IllegalArgumentException(s"Unsupported storage format: '$fileFormat'") + } + +} diff --git a/src/main/scala/com/exasol/cloudetl/storage/FileFormat.scala b/src/main/scala/com/exasol/cloudetl/storage/FileFormat.scala index 8f79d52d..68b21d71 100644 --- a/src/main/scala/com/exasol/cloudetl/storage/FileFormat.scala +++ b/src/main/scala/com/exasol/cloudetl/storage/FileFormat.scala @@ -16,13 +16,14 @@ object FileFormat { case _ => throw new IllegalArgumentException(s"Unsupported file format $fileFormat!") } + case object AVRO extends FileFormat + case object FILE extends FileFormat + case object ORC extends FileFormat + case object PARQUET extends FileFormat + } /** * An enum for supported file formats. */ sealed trait FileFormat extends Product with Serializable -case object AVRO extends FileFormat -case object FILE extends FileFormat -case object ORC extends FileFormat -case object PARQUET extends FileFormat diff --git a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala index 79adf55b..32fa0011 100644 --- a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala @@ -62,9 +62,14 @@ object StorageProperties { */ private[storage] final val KEY_VALUE_SEPARATOR: String = " -> " - val BUCKET_PATH: String = "BUCKET_PATH" - val DATA_FORMAT: String = "DATA_FORMAT" - val PARALLELISM: String = "PARALLELISM" + /** A required property key name for a bucket path. */ + private[storage] final val BUCKET_PATH: String = "BUCKET_PATH" + + /** A required property key name for a data format. */ + private[storage] final val DATA_FORMAT: String = "DATA_FORMAT" + + /** An optional property key name for the parallelism. */ + private[storage] final val PARALLELISM: String = "PARALLELISM" def apply(params: Map[String, String]): StorageProperties = new StorageProperties(params) diff --git a/src/test/scala/com/exasol/cloudetl/TestUtils.scala b/src/test/scala/com/exasol/cloudetl/DummyRecordsTest.scala similarity index 98% rename from src/test/scala/com/exasol/cloudetl/TestUtils.scala rename to src/test/scala/com/exasol/cloudetl/DummyRecordsTest.scala index ac842cf6..ab075b38 100644 --- a/src/test/scala/com/exasol/cloudetl/TestUtils.scala +++ b/src/test/scala/com/exasol/cloudetl/DummyRecordsTest.scala @@ -7,7 +7,7 @@ import java.nio.file.attribute.BasicFileAttributes import java.sql.Date import java.sql.Timestamp -trait TestUtils { +trait DummyRecordsTest { val BIG_DECIMAL_VALUE1: BigDecimal = new BigDecimal("5555555555555555555555555555555.55555") val BIG_DECIMAL_VALUE2: BigDecimal = new BigDecimal("5555555555555555555555555555555.55555") diff --git a/src/test/scala/com/exasol/cloudetl/bucket/BucketSuite.scala b/src/test/scala/com/exasol/cloudetl/bucket/BucketTest.scala similarity index 67% rename from src/test/scala/com/exasol/cloudetl/bucket/BucketSuite.scala rename to src/test/scala/com/exasol/cloudetl/bucket/BucketTest.scala index 859f954d..ae536d17 100644 --- a/src/test/scala/com/exasol/cloudetl/bucket/BucketSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/bucket/BucketTest.scala @@ -5,34 +5,44 @@ import org.apache.hadoop.fs.azure.NativeAzureFileSystem import org.apache.hadoop.fs.azure.Wasb import org.apache.hadoop.fs.azure.Wasbs import org.apache.hadoop.fs.s3a.S3AFileSystem +import org.scalatest.BeforeAndAfterEach import org.scalatest.FunSuite -import org.scalatest.Matchers @SuppressWarnings(Array("org.wartremover.warts.IsInstanceOf")) -class BucketSuite extends FunSuite with Matchers { +class BucketTest extends FunSuite with BeforeAndAfterEach { - test("throws an exception if the scheme is not supported") { + private[this] val PATH: String = "BUCKET_PATH" + private[this] val FORMAT: String = "DATA_FORMAT" + private[this] var properties: Map[String, String] = _ + + override final def beforeEach(): Unit = { + properties = Map.empty[String, String] + () + } + + test("apply throws if the scheme is not supported") { + properties = Map(PATH -> "xyz:/bucket/files*", FORMAT -> "ORC") val thrown = intercept[IllegalArgumentException] { - Bucket(Map(Bucket.BUCKET_PATH -> "xyz:/bucket/files*")) + Bucket(properties) } - assert(thrown.getMessage === "Unsupported path scheme xyz") + assert(thrown.getMessage === "Unsupported path scheme xyz!") } - test("creates an LocalBucket with local path parameter") { - val bucket = Bucket(Map(Bucket.BUCKET_PATH -> "file://local/path/bucket/")) + test("apply returns LocalBucket") { + properties = Map(PATH -> "file://local/path/bucket/", FORMAT -> "ORC") + val bucket = Bucket(properties) assert(bucket.isInstanceOf[LocalBucket]) } - test("creates an S3Bucket with given parameters") { - val s3params = Map( - Bucket.BUCKET_PATH -> "s3a://my-bucket/", - "DATA_FORMAT" -> "PARQUET", + test("apply returns S3Bucket") { + properties = Map( + PATH -> "s3a://my-bucket/", + FORMAT -> "ORC", "S3_ENDPOINT" -> "eu-central-1", "S3_ACCESS_KEY" -> "abc", "S3_SECRET_KEY" -> "xyz" ) - - val bucket = Bucket(s3params) + val bucket = Bucket(properties) val conf = bucket.getConfiguration() assert(bucket.isInstanceOf[S3Bucket]) @@ -42,15 +52,14 @@ class BucketSuite extends FunSuite with Matchers { assert(conf.get("fs.s3a.secret.key") === "xyz") } - test("creates a GCSBucket with given parameters") { - val gcsParams = Map( - Bucket.BUCKET_PATH -> "gs://my-bucket/", - "DATA_FORMAT" -> "AVRO", + test("apply returns GCSBucket") { + properties = Map( + PATH -> "gs://my-bucket/", + FORMAT -> "AVRO", "GCS_PROJECT_ID" -> "projX", "GCS_KEYFILE_PATH" -> "/bucketfs/bucket1/projX.json" ) - - val bucket = Bucket(gcsParams) + val bucket = Bucket(properties) val conf = bucket.getConfiguration() assert(bucket.isInstanceOf[GCSBucket]) @@ -59,15 +68,14 @@ class BucketSuite extends FunSuite with Matchers { assert(conf.get("fs.gs.auth.service.account.json.keyfile") === "/bucketfs/bucket1/projX.json") } - test("creates an AzureBlobBucket with given parameters") { - val azureBlobParams = Map( - Bucket.BUCKET_PATH -> "wasbs://container@account1/parquet-bucket/", - "DATA_FORMAT" -> "AVRO", + test("apply returns AzureBlobBucket") { + properties = Map( + PATH -> "wasbs://container@account1/parquet-bucket/", + FORMAT -> "AVRO", "AZURE_ACCOUNT_NAME" -> "account1", "AZURE_SECRET_KEY" -> "secret" ) - - val bucket = Bucket(azureBlobParams) + val bucket = Bucket(properties) val conf = bucket.getConfiguration() assert(bucket.isInstanceOf[AzureBlobBucket]) @@ -85,16 +93,15 @@ class BucketSuite extends FunSuite with Matchers { } } - test("creates an AzureAdlsBucket with provided parameters") { - val params = Map( - Bucket.BUCKET_PATH -> "adl://my_container.azuredatalakestore.net/orc/*", - "DATA_FORMAT" -> "CSV", + test("apply returns AzureAdlsBucket") { + properties = Map( + PATH -> "adl://my_container.azuredatalakestore.net/orc/*", + FORMAT -> "CSV", "AZURE_CLIENT_ID" -> "clientX", "AZURE_CLIENT_SECRET" -> "client_secret", "AZURE_DIRECTORY_ID" -> "directory_id_secret" ) - - val bucket = Bucket(params) + val bucket = Bucket(properties) assert(bucket.isInstanceOf[AzureAdlsBucket]) val conf = bucket.getConfiguration() diff --git a/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala index 15ba6496..105b97bd 100644 --- a/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala +++ b/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala @@ -3,7 +3,6 @@ package com.exasol.cloudetl.common import org.scalatest.BeforeAndAfterEach import org.scalatest.FunSuite -@SuppressWarnings(Array("org.wartremover.warts.Var")) class AbstractPropertiesTest extends FunSuite with BeforeAndAfterEach { private[this] var properties: Map[String, String] = _ diff --git a/src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerSuite.scala b/src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerTest.scala similarity index 73% rename from src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerSuite.scala rename to src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerTest.scala index 9a2241ed..339abb7f 100644 --- a/src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerTest.scala @@ -2,11 +2,10 @@ package com.exasol.cloudetl.orc import org.apache.orc.TypeDescription import org.scalatest.FunSuite -import org.scalatest.Matchers -class OrcDeserializerSuite extends FunSuite with Matchers { +class OrcDeserializerTest extends FunSuite { - test("throws IllegalArgumentException when orc struct type contains a list") { + test("apply throws if orc type is a list") { val orcList = TypeDescription.createList(TypeDescription.createString) val thrown = intercept[IllegalArgumentException] { OrcDeserializer(orcList) @@ -14,7 +13,7 @@ class OrcDeserializerSuite extends FunSuite with Matchers { assert(thrown.getMessage === "Orc list type is not supported.") } - test("throws IllegalArgumentException when orc struct type contains a map") { + test("apply throws f orc type is a map") { val orcMap = TypeDescription.createMap(TypeDescription.createString, TypeDescription.createString) val thrown = intercept[IllegalArgumentException] { @@ -23,7 +22,7 @@ class OrcDeserializerSuite extends FunSuite with Matchers { assert(thrown.getMessage === "Orc map type is not supported.") } - test("throws IllegalArgumentException when orc struct type contains a nested struct") { + test("apply throws if orc type is a nested struct") { val orcStruct = TypeDescription.createStruct().addField("col_int", TypeDescription.createInt()) val thrown = intercept[IllegalArgumentException] { @@ -32,7 +31,7 @@ class OrcDeserializerSuite extends FunSuite with Matchers { assert(thrown.getMessage === "Orc nested struct type is not supported.") } - test("throws IllegalArgumentException for unsupported type") { + test("apply throws if orc type is unsupported") { val orcUnion = TypeDescription.createUnion() val thrown = intercept[IllegalArgumentException] { OrcDeserializer(orcUnion) diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/BaseSuite.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/BaseSuite.scala deleted file mode 100644 index 52450fdf..00000000 --- a/src/test/scala/com/exasol/cloudetl/scriptclasses/BaseSuite.scala +++ /dev/null @@ -1,51 +0,0 @@ -package com.exasol.cloudetl.scriptclasses - -import java.nio.file.Path -import java.nio.file.Paths - -import com.exasol.ExaIterator -import com.exasol.cloudetl.bucket.Bucket - -import org.mockito.Mockito.when -import org.scalatest.FunSuite -import org.scalatest.Matchers -import org.scalatest.mockito.MockitoSugar - -trait BaseSuite extends FunSuite with Matchers with MockitoSugar { - - val testSchema = "my_schema" - - val s3BucketPath = "s3a://my_bucket/folder1/*" - val s3Endpoint = "s3.eu-central-1.com" - val s3AccessKey = "s3_access_key" - val s3SecretKey = "s3_secret_key" - - val params: Map[String, String] = Map( - "BUCKET_PATH" -> s3BucketPath, - "DATA_FORMAT" -> "PARQUET", - "S3_ENDPOINT" -> s3Endpoint, - "S3_ACCESS_KEY" -> s3AccessKey, - "S3_SECRET_KEY" -> s3SecretKey - ) - - val rest = - s"""BUCKET_PATH:=:$s3BucketPath;DATA_FORMAT:=:PARQUET;S3_ACCESS_KEY:=:$s3AccessKey;""" + - s"""S3_ENDPOINT:=:$s3Endpoint;S3_SECRET_KEY:=:$s3SecretKey""" - - val resourcePath: String = norm(Paths.get(getClass.getResource("/data").toURI)) - val resourceImportBucket: String = s"$resourcePath/import/parquet/sales_pos*.parquet" - - final def norm(path: Path): String = - path.toUri.toString.replaceAll("/$", "").replaceAll("///", "/") - - @SuppressWarnings(Array("org.wartremover.warts.DefaultArguments")) - final def commonExaIterator(bucket: String, format: String = "PARQUET"): ExaIterator = { - val mockIter = mock[ExaIterator] - val newParams = params + (Bucket.BUCKET_PATH -> bucket, "DATA_FORMAT" -> format) - - when(mockIter.getString(0)).thenReturn(bucket) - when(mockIter.getString(1)).thenReturn(Bucket.keyValueMapToString(newParams)) - - mockIter - } -} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathSuite.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathSuite.scala deleted file mode 100644 index 581bdbcf..00000000 --- a/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathSuite.scala +++ /dev/null @@ -1,77 +0,0 @@ -package com.exasol.cloudetl.scriptclasses - -import scala.collection.JavaConverters._ - -import com.exasol.ExaExportSpecification -import com.exasol.ExaMetadata - -import org.mockito.Mockito._ - -class ExportPathSuite extends BaseSuite { - - test("`generateSqlForExportSpec` should create a sql statement") { - val exaMeta = mock[ExaMetadata] - val exaSpec = mock[ExaExportSpecification] - - when(exaMeta.getScriptSchema()).thenReturn(testSchema) - when(exaSpec.getParameters()).thenReturn(params.asJava) - - val srcCols = Seq("tbl.col_int", "c_bool", "c_char") - when(exaSpec.getSourceColumnNames).thenReturn(srcCols.asJava) - - val sqlExpected = - s"""SELECT - | $testSchema.EXPORT_TABLE( - | '$s3BucketPath', '$rest', 'col_int.c_bool.c_char', col_int, c_bool, c_char - |) - |FROM - | DUAL - |GROUP BY - | iproc(); - |""".stripMargin - - assert(ExportPath.generateSqlForExportSpec(exaMeta, exaSpec) === sqlExpected) - verify(exaMeta, atLeastOnce).getScriptSchema - verify(exaSpec, times(1)).getParameters - verify(exaSpec, times(1)).getSourceColumnNames - } - - test("`generateSqlForExportSpec` should throw an exception if any required param is missing") { - val exaMeta = mock[ExaMetadata] - val exaSpec = mock[ExaExportSpecification] - - val newParams = params - ("S3_ACCESS_KEY") - - when(exaMeta.getScriptSchema()).thenReturn(testSchema) - when(exaSpec.getParameters()).thenReturn(newParams.asJava) - - val thrown = intercept[IllegalArgumentException] { - ExportPath.generateSqlForExportSpec(exaMeta, exaSpec) - } - - assert(thrown.getMessage === "Please provide a value for the S3_ACCESS_KEY property!") - verify(exaSpec, times(1)).getParameters - verify(exaSpec, never).getSourceColumnNames - } - - test("`generateSqlForExportSpec` throws if column cannot be parsed (contains extra '.')") { - val exaMeta = mock[ExaMetadata] - val exaSpec = mock[ExaExportSpecification] - - when(exaMeta.getScriptSchema()).thenReturn(testSchema) - when(exaSpec.getParameters()).thenReturn(params.asJava) - - val srcCols = Seq("tbl.c_int.integer") - when(exaSpec.getSourceColumnNames).thenReturn(srcCols.asJava) - - val thrown = intercept[RuntimeException] { - ExportPath.generateSqlForExportSpec(exaMeta, exaSpec) - } - - assert(thrown.getMessage === "Could not parse the column name from 'tbl.c_int.integer'!") - verify(exaMeta, atLeastOnce).getScriptSchema - verify(exaSpec, times(1)).getParameters - verify(exaSpec, times(1)).getSourceColumnNames - } - -} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathTest.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathTest.scala new file mode 100644 index 00000000..2307d95d --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportPathTest.scala @@ -0,0 +1,66 @@ +package com.exasol.cloudetl.scriptclasses + +import scala.collection.JavaConverters._ + +import com.exasol.cloudetl.storage.StorageProperties + +import org.mockito.Mockito._ + +class ExportPathTest extends PathTest { + + test("generateSqlForExportSpec returns SQL statement") { + when(metadata.getScriptSchema()).thenReturn(schema) + when(exportSpec.getParameters()).thenReturn(properties.asJava) + val srcCols = Seq("tbl.col_int", "c_bool", "c_char") + when(exportSpec.getSourceColumnNames).thenReturn(srcCols.asJava) + + val storageProperties = StorageProperties(properties) + val bucketPath = storageProperties.getStoragePath() + val stringPairs = storageProperties.mkString() + + val expectedSQLStatement = + s"""SELECT + | $schema.EXPORT_TABLE( + | '$bucketPath', '$stringPairs', 'col_int.c_bool.c_char', col_int, c_bool, c_char + |) + |FROM + | DUAL + |GROUP BY + | iproc(); + |""".stripMargin + + assert(ExportPath.generateSqlForExportSpec(metadata, exportSpec) === expectedSQLStatement) + verify(metadata, atLeastOnce).getScriptSchema + verify(exportSpec, times(1)).getParameters + verify(exportSpec, times(1)).getSourceColumnNames + } + + test("generateSqlForExportSpec throws if required property is not set") { + val newProperties = properties - ("S3_ACCESS_KEY") + when(metadata.getScriptSchema()).thenReturn(schema) + when(exportSpec.getParameters()).thenReturn(newProperties.asJava) + + val thrown = intercept[IllegalArgumentException] { + ExportPath.generateSqlForExportSpec(metadata, exportSpec) + } + assert(thrown.getMessage === "Please provide a value for the S3_ACCESS_KEY property!") + verify(exportSpec, times(1)).getParameters + verify(exportSpec, never).getSourceColumnNames + } + + test("generateSqlForExportSpec throws if columns cannot be parsed (e.g, contains extra '.')") { + when(metadata.getScriptSchema()).thenReturn(schema) + when(exportSpec.getParameters()).thenReturn(properties.asJava) + val srcCols = Seq("tbl.c_int.integer") + when(exportSpec.getSourceColumnNames).thenReturn(srcCols.asJava) + + val thrown = intercept[RuntimeException] { + ExportPath.generateSqlForExportSpec(metadata, exportSpec) + } + assert(thrown.getMessage === "Could not parse the column name from 'tbl.c_int.integer'!") + verify(metadata, atLeastOnce).getScriptSchema + verify(exportSpec, times(1)).getParameters + verify(exportSpec, times(1)).getSourceColumnNames + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportTableSuite.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportTableTest.scala similarity index 63% rename from src/test/scala/com/exasol/cloudetl/scriptclasses/ExportTableSuite.scala rename to src/test/scala/com/exasol/cloudetl/scriptclasses/ExportTableTest.scala index 76e46f21..0324b5ea 100644 --- a/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportTableSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/ExportTableTest.scala @@ -4,20 +4,17 @@ import java.nio.file.Path import com.exasol.ExaIterator import com.exasol.ExaMetadata -import com.exasol.cloudetl.TestUtils +import com.exasol.cloudetl.DummyRecordsTest import org.mockito.ArgumentMatchers.any import org.mockito.ExtraMockito import org.mockito.Mockito._ import org.scalatest.BeforeAndAfterEach -@SuppressWarnings(Array("org.wartremover.warts.JavaSerializable", "org.wartremover.warts.Var")) -class ExportTableSuite extends BaseSuite with BeforeAndAfterEach with TestUtils { +class ExportTableTest extends StorageTest with DummyRecordsTest with BeforeAndAfterEach { - private var outputPath: Path = _ - private var exasolMetadata: ExaMetadata = _ - private var exasolIterator: ExaIterator = _ - private val srcColumns: Seq[String] = Seq( + private[this] var outputPath: Path = _ + private[this] val srcColumns: Seq[String] = Seq( "c_int", "c_long", "c_decimal", @@ -28,8 +25,12 @@ class ExportTableSuite extends BaseSuite with BeforeAndAfterEach with TestUtils "c_timestamp" ) + private[this] var metadata: ExaMetadata = _ + private[this] var iterator: ExaIterator = _ + final def createMockedIterator(resourceDir: String): ExaIterator = { - val mockedIterator = commonExaIterator(resourceDir) + val properties = Map("BUCKET_PATH" -> resourceDir, "DATA_FORMAT" -> "PARQUET") + val mockedIterator = mockExasolIterator(properties) when(mockedIterator.getString(2)).thenReturn(srcColumns.mkString(".")) when(mockedIterator.next()).thenReturn(true, false) when(mockedIterator.size()).thenReturn(2L) @@ -46,6 +47,7 @@ class ExportTableSuite extends BaseSuite with BeforeAndAfterEach with TestUtils mockedIterator } + @SuppressWarnings(Array("org.wartremover.warts.JavaSerializable")) final def createMockedMetadata(): ExaMetadata = { val mockedMetadata = mock[ExaMetadata] when(mockedMetadata.getInputColumnCount()).thenReturn(11L) @@ -72,8 +74,8 @@ class ExportTableSuite extends BaseSuite with BeforeAndAfterEach with TestUtils override final def beforeEach(): Unit = { outputPath = createTemporaryFolder("exportTableTest") - exasolMetadata = createMockedMetadata() - exasolIterator = createMockedIterator(outputPath.toUri.toString) + metadata = createMockedMetadata() + iterator = createMockedIterator(outputPath.toUri.toString) () } @@ -82,31 +84,32 @@ class ExportTableSuite extends BaseSuite with BeforeAndAfterEach with TestUtils () } - test("`run` should export the Exasol rows from ExaIterator") { - ExportTable.run(exasolMetadata, exasolIterator) + test("run exports table rows") { + ExportTable.run(metadata, iterator) - verify(exasolMetadata, times(1)).getInputColumnCount + verify(metadata, times(1)).getInputColumnCount for { idx <- 3 to 10 } { - verify(exasolMetadata, times(1)).getInputColumnType(idx) - verify(exasolMetadata, times(1)).getInputColumnPrecision(idx) - verify(exasolMetadata, times(1)).getInputColumnScale(idx) - verify(exasolMetadata, times(1)).getInputColumnLength(idx) + verify(metadata, times(1)).getInputColumnType(idx) + verify(metadata, times(1)).getInputColumnPrecision(idx) + verify(metadata, times(1)).getInputColumnScale(idx) + verify(metadata, times(1)).getInputColumnLength(idx) } - verify(exasolIterator, times(2)).getInteger(3) - verify(exasolIterator, times(2)).getLong(4) - verify(exasolIterator, times(2)).getBigDecimal(5) - verify(exasolIterator, times(2)).getDouble(6) - verify(exasolIterator, times(2)).getString(7) - verify(exasolIterator, times(2)).getBoolean(8) - verify(exasolIterator, times(2)).getDate(9) - verify(exasolIterator, times(2)).getTimestamp(10) + verify(iterator, times(2)).getInteger(3) + verify(iterator, times(2)).getLong(4) + verify(iterator, times(2)).getBigDecimal(5) + verify(iterator, times(2)).getDouble(6) + verify(iterator, times(2)).getString(7) + verify(iterator, times(2)).getBoolean(8) + verify(iterator, times(2)).getDate(9) + verify(iterator, times(2)).getTimestamp(10) } - test("import exported rows from a file") { - ExportTable.run(exasolMetadata, exasolIterator) + test("imports exported rows from a path") { + ExportTable.run(metadata, iterator) - val importIter = commonExaIterator(resourceImportBucket) + val properties = Map("BUCKET_PATH" -> testResourceDir, "DATA_FORMAT" -> "PARQUET") + val importIter = mockExasolIterator(properties) when(importIter.next()).thenReturn(false) when(importIter.getString(2)).thenReturn(outputPath.toUri.toString) diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportFilesSuite.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportFilesSuite.scala deleted file mode 100644 index a1ed7b7b..00000000 --- a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportFilesSuite.scala +++ /dev/null @@ -1,93 +0,0 @@ -package com.exasol.cloudetl.scriptclasses - -import com.exasol.ExaIterator -import com.exasol.ExaMetadata - -import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito._ - -class ImportFilesSuite extends BaseSuite { - - test("`run` should throw if the source is not supported") { - val file = s"$resourcePath/import/parquet/sales_positions1.snappy.parquet" - val exaIter = commonExaIterator(resourceImportBucket, "CSV") - when(exaIter.next()).thenReturn(false) - when(exaIter.getString(2)).thenReturn(file) - - val thrown = intercept[IllegalArgumentException] { - ImportFiles.run(mock[ExaMetadata], exaIter) - } - assert(thrown.getMessage === "Unsupported storage format: 'CSV'") - } - - test("`run` should emit total number of records from a source") { - val file1 = s"$resourcePath/import/parquet/sales_positions1.snappy.parquet" - val file2 = s"$resourcePath/import/parquet/sales_positions2.snappy.parquet" - - val exaIter = commonExaIterator(resourceImportBucket) - when(exaIter.next()).thenReturn(true, false) - when(exaIter.getString(2)).thenReturn(file1, file2) - - ImportFiles.run(mock[ExaMetadata], exaIter) - - val totalRecords = 1000 - verify(exaIter, times(totalRecords)).emit(Seq(any[Object]): _*) - } - - /** - * - * +---------+-----------+----------+------+-----+----------+--------+ - * |sales_id |position_id|article_id|amount|price|voucher_id|canceled| - * +---------+-----------+----------+------+-----+----------+--------+ - * |582244536|2 |96982 |1 |0.56 |null |null | - * |582177839|6 |96982 |2 |0.56 |null |null | - * |582370207|0 |96982 |1 |0.56 |null |null | - * |582344312|0 |96982 |5 |0.56 |null |null | - * |582344274|1 |96982 |1 |0.56 |null |null | - * +---------+-----------+----------+------+-----+----------+--------+ - * - */ - test("`run` should emit correct sequence of records from parquet file") { - val file = s"$resourcePath/import/parquet/sales_positions_small.snappy.parquet" - - val exaIter = commonExaIterator(resourceImportBucket) - when(exaIter.next()).thenReturn(false) - when(exaIter.getString(2)).thenReturn(file) - - ImportFiles.run(mock[ExaMetadata], exaIter) - - verifySmallFilesImport(exaIter) - } - - test("`run` should emit correct sequence of records from avro file") { - val file = s"$resourcePath/import/avro/sales_positions_small.avro" - - val exaIter = commonExaIterator(resourceImportBucket, "AVRO") - when(exaIter.next()).thenReturn(false) - when(exaIter.getString(2)).thenReturn(file) - - ImportFiles.run(mock[ExaMetadata], exaIter) - - verifySmallFilesImport(exaIter) - } - - private def verifySmallFilesImport(iter: ExaIterator): Unit = { - val totalRecords = 5 - val records: Seq[Seq[Object]] = Seq( - Seq(582244536L, 2, 96982, 1, 0.56, null, null), - Seq(582177839L, 6, 96982, 2, 0.56, null, null), - Seq(582370207L, 0, 96982, 1, 0.56, null, null), - Seq(582344312L, 0, 96982, 5, 0.56, null, null), - Seq(582344274L, 1, 96982, 1, 0.56, null, null) - ).map { seq => - seq.map(_.asInstanceOf[AnyRef]) - } - - verify(iter, times(totalRecords)).emit(Seq(any[Object]): _*) - records.foreach { - case rows => - verify(iter, times(1)).emit(rows: _*) - } - } - -} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportFilesTest.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportFilesTest.scala new file mode 100644 index 00000000..9052fa2d --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportFilesTest.scala @@ -0,0 +1,92 @@ +package com.exasol.cloudetl.scriptclasses + +import com.exasol.ExaIterator +import com.exasol.ExaMetadata + +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito._ + +class ImportFilesTest extends StorageTest { + + private[this] val properties = Map( + "BUCKET_PATH" -> testResourceParquetPath, + "DATA_FORMAT" -> "PARQUET" + ) + + test("run throws if file format is not supported") { + val file = s"$testResourceDir/import/parquet/sales_positions1.snappy.parquet" + val iter = mockFileIterator("CSV", file) + + val thrown = intercept[IllegalArgumentException] { + ImportFiles.run(mock[ExaMetadata], iter) + } + assert(thrown.getMessage === "Unsupported file format CSV!") + } + + test("run emits all records from a source") { + val file1 = s"$testResourceDir/import/parquet/sales_positions1.snappy.parquet" + val file2 = s"$testResourceDir/import/parquet/sales_positions2.snappy.parquet" + val expectedNumberOfRecords = 1000 + + val iter = mockExasolIterator(properties) + when(iter.next()).thenReturn(true, false) + when(iter.getString(2)).thenReturn(file1, file2) + + ImportFiles.run(mock[ExaMetadata], iter) + verify(iter, times(expectedNumberOfRecords)).emit(Seq(any[Object]): _*) + } + + /** + * + * +---------+-----------+----------+------+-----+----------+--------+ + * |sales_id |position_id|article_id|amount|price|voucher_id|canceled| + * +---------+-----------+----------+------+-----+----------+--------+ + * |582244536|2 |96982 |1 |0.56 |null |null | + * |582177839|6 |96982 |2 |0.56 |null |null | + * |582370207|0 |96982 |1 |0.56 |null |null | + * |582344312|0 |96982 |5 |0.56 |null |null | + * |582344274|1 |96982 |1 |0.56 |null |null | + * +---------+-----------+----------+------+-----+----------+--------+ + * + */ + test("run emits correct sequence of records from PARQUET file") { + val parquetFile = s"$testResourceDir/import/parquet/sales_positions_small.snappy.parquet" + val iter = mockFileIterator("PARQUET", parquetFile) + ImportFiles.run(mock[ExaMetadata], iter) + verifySmallFilesImport(iter) + } + + test("run emits correct sequence of records from AVRO file") { + val avroFile = s"$testResourceDir/import/avro/sales_positions_small.avro" + val iter = mockFileIterator("AVRO", avroFile) + ImportFiles.run(mock[ExaMetadata], iter) + verifySmallFilesImport(iter) + } + + private[this] def mockFileIterator(fileFormat: String, filename: String): ExaIterator = { + val iter = mockExasolIterator(properties ++ Map("DATA_FORMAT" -> fileFormat)) + when(iter.next()).thenReturn(false) + when(iter.getString(2)).thenReturn(filename) + iter + } + + private[this] def verifySmallFilesImport(iter: ExaIterator): Unit = { + val totalRecords = 5 + val records: Seq[Seq[Object]] = Seq( + Seq(582244536L, 2, 96982, 1, 0.56, null, null), + Seq(582177839L, 6, 96982, 2, 0.56, null, null), + Seq(582370207L, 0, 96982, 1, 0.56, null, null), + Seq(582344312L, 0, 96982, 5, 0.56, null, null), + Seq(582344274L, 1, 96982, 1, 0.56, null, null) + ).map { seq => + seq.map(_.asInstanceOf[AnyRef]) + } + + verify(iter, times(totalRecords)).emit(Seq(any[Object]): _*) + records.foreach { + case rows => + verify(iter, times(1)).emit(rows: _*) + } + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportMetadataSuite.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportMetadataSuite.scala deleted file mode 100644 index 35193c20..00000000 --- a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportMetadataSuite.scala +++ /dev/null @@ -1,25 +0,0 @@ -package com.exasol.cloudetl.scriptclasses - -import com.exasol.ExaMetadata - -import org.mockito.ArgumentMatchers.anyString -import org.mockito.Mockito._ - -class ImportMetadataSuite extends BaseSuite { - - test("`run` should create a list of files names") { - val exaIter = commonExaIterator(resourceImportBucket) - when(exaIter.getInteger(2)).thenReturn(2) - - ImportMetadata.run(mock[ExaMetadata], exaIter) - - verify(exaIter, times(3)).emit(anyString(), anyString()) - verify(exaIter, times(1)) - .emit(s"$resourcePath/import/parquet/sales_positions1.snappy.parquet", "0") - verify(exaIter, times(1)) - .emit(s"$resourcePath/import/parquet/sales_positions2.snappy.parquet", "1") - verify(exaIter, times(1)) - .emit(s"$resourcePath/import/parquet/sales_positions_small.snappy.parquet", "0") - } - -} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportMetadataTest.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportMetadataTest.scala new file mode 100644 index 00000000..0adc34b8 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportMetadataTest.scala @@ -0,0 +1,32 @@ +package com.exasol.cloudetl.scriptclasses + +import com.exasol.ExaMetadata + +import org.mockito.ArgumentMatchers.anyString +import org.mockito.Mockito._ + +class ImportMetadataTest extends StorageTest { + + test("run returns the list of file names") { + val properties = Map( + "BUCKET_PATH" -> testResourceParquetPath, + "DATA_FORMAT" -> "PARQUET" + ) + val expectedParquetFiles = Map( + s"$testResourceDir/import/parquet/sales_positions1.snappy.parquet" -> "0", + s"$testResourceDir/import/parquet/sales_positions2.snappy.parquet" -> "1", + s"$testResourceDir/import/parquet/sales_positions_small.snappy.parquet" -> "0" + ) + + val iter = mockExasolIterator(properties) + when(iter.getInteger(2)).thenReturn(2) + + ImportMetadata.run(mock[ExaMetadata], iter) + verify(iter, times(3)).emit(anyString(), anyString()) + expectedParquetFiles.foreach { + case (filename, partitionId) => + verify(iter, times(1)).emit(filename, partitionId) + } + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathSuite.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathSuite.scala deleted file mode 100644 index 325557ff..00000000 --- a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathSuite.scala +++ /dev/null @@ -1,55 +0,0 @@ -package com.exasol.cloudetl.scriptclasses - -import scala.collection.JavaConverters._ - -import com.exasol.ExaImportSpecification -import com.exasol.ExaMetadata - -import org.mockito.Mockito._ - -class ImportPathSuite extends BaseSuite { - - test("`generateSqlForImportSpec` should create a sql statement") { - val exaMeta = mock[ExaMetadata] - val exaSpec = mock[ExaImportSpecification] - - when(exaMeta.getScriptSchema()).thenReturn(testSchema) - when(exaSpec.getParameters()).thenReturn(params.asJava) - - val sqlExpected = - s"""SELECT - | $testSchema.IMPORT_FILES( - | '$s3BucketPath', '$rest', filename - |) - |FROM ( - | SELECT $testSchema.IMPORT_METADATA( - | '$s3BucketPath', '$rest', nproc() - | ) - |) - |GROUP BY - | partition_index; - |""".stripMargin - - assert(ImportPath.generateSqlForImportSpec(exaMeta, exaSpec) === sqlExpected) - verify(exaMeta, atLeastOnce).getScriptSchema - verify(exaSpec, times(1)).getParameters - } - - test("`generateSqlForImportSpec` should throw an exception if any required param is missing") { - val exaMeta = mock[ExaMetadata] - val exaSpec = mock[ExaImportSpecification] - - val newParams = params - ("S3_ACCESS_KEY") - - when(exaMeta.getScriptSchema()).thenReturn(testSchema) - when(exaSpec.getParameters()).thenReturn(newParams.asJava) - - val thrown = intercept[IllegalArgumentException] { - ImportPath.generateSqlForImportSpec(exaMeta, exaSpec) - } - - assert(thrown.getMessage === "Please provide a value for the S3_ACCESS_KEY property!") - verify(exaSpec, times(1)).getParameters - } - -} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathTest.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathTest.scala new file mode 100644 index 00000000..da29db6a --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/ImportPathTest.scala @@ -0,0 +1,50 @@ +package com.exasol.cloudetl.scriptclasses + +import scala.collection.JavaConverters._ + +import com.exasol.cloudetl.storage.StorageProperties + +import org.mockito.Mockito._ + +class ImportPathTest extends PathTest { + + test("generateSqlForImportSpec returns SQL statement") { + when(metadata.getScriptSchema()).thenReturn(schema) + when(importSpec.getParameters()).thenReturn(properties.asJava) + + val storageProperties = StorageProperties(properties) + val bucketPath = storageProperties.getStoragePath() + val storagePropertyPairs = storageProperties.mkString() + + val expectedSQLStatement = + s"""SELECT + | $schema.IMPORT_FILES( + | '$bucketPath', '$storagePropertyPairs', filename + |) + |FROM ( + | SELECT $schema.IMPORT_METADATA( + | '$bucketPath', '$storagePropertyPairs', nproc() + | ) + |) + |GROUP BY + | partition_index; + |""".stripMargin + + assert(ImportPath.generateSqlForImportSpec(metadata, importSpec) === expectedSQLStatement) + verify(metadata, atLeastOnce).getScriptSchema + verify(importSpec, times(1)).getParameters + } + + test("generateSqlForImportSpec throws if required property is not set") { + val newProperties = properties - ("S3_ACCESS_KEY") + when(metadata.getScriptSchema()).thenReturn(schema) + when(importSpec.getParameters()).thenReturn(newProperties.asJava) + + val thrown = intercept[IllegalArgumentException] { + ImportPath.generateSqlForImportSpec(metadata, importSpec) + } + assert(thrown.getMessage === "Please provide a value for the S3_ACCESS_KEY property!") + verify(importSpec, times(1)).getParameters + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/PathTest.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/PathTest.scala new file mode 100644 index 00000000..61f34fbc --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/PathTest.scala @@ -0,0 +1,37 @@ +package com.exasol.cloudetl.scriptclasses + +import com.exasol.ExaExportSpecification +import com.exasol.ExaImportSpecification +import com.exasol.ExaMetadata + +import org.scalatest.BeforeAndAfterEach +import org.scalatest.FunSuite +import org.scalatest.mockito.MockitoSugar + +/** + * A trait with helper variables for import or export path test classes. + */ +trait PathTest extends FunSuite with BeforeAndAfterEach with MockitoSugar { + + private[scriptclasses] val schema = "myDBSchema" + + private[scriptclasses] val properties = Map( + "BUCKET_PATH" -> "s3a://my_bucket/folder1/*", + "DATA_FORMAT" -> "PARQUET", + "S3_ENDPOINT" -> "s3.eu-central-1.com", + "S3_ACCESS_KEY" -> "s3_access_key", + "S3_SECRET_KEY" -> "s3_secret_key" + ) + + private[scriptclasses] var metadata: ExaMetadata = _ + private[scriptclasses] var importSpec: ExaImportSpecification = _ + private[scriptclasses] var exportSpec: ExaExportSpecification = _ + + override final def beforeEach(): Unit = { + metadata = mock[ExaMetadata] + importSpec = mock[ExaImportSpecification] + exportSpec = mock[ExaExportSpecification] + () + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/StorageTest.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/StorageTest.scala new file mode 100644 index 00000000..773d8e14 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/StorageTest.scala @@ -0,0 +1,38 @@ +package com.exasol.cloudetl.scriptclasses + +import java.nio.file.Path +import java.nio.file.Paths + +import com.exasol.ExaIterator +import com.exasol.cloudetl.storage.StorageProperties + +import org.mockito.Mockito.when +import org.scalatest.FunSuite +import org.scalatest.mockito.MockitoSugar + +/** + * A trait with helper methods and default values for storage test + * classes. + */ +trait StorageTest extends FunSuite with MockitoSugar { + + private[scriptclasses] val testResourceDir: String = + normalize(Paths.get(getClass.getResource("/data").toURI)) + + private[scriptclasses] val testResourceParquetPath: String = + s"$testResourceDir/import/parquet/sales_pos*.parquet" + + final def normalize(path: Path): String = + path.toUri.toString.replaceAll("/$", "").replaceAll("///", "/") + + final def mockExasolIterator(params: Map[String, String]): ExaIterator = { + val storageProperties = StorageProperties(params) + val bucketPath = storageProperties.getStoragePath() + + val mockedIterator = mock[ExaIterator] + when(mockedIterator.getString(0)).thenReturn(bucketPath) + when(mockedIterator.getString(1)).thenReturn(storageProperties.mkString()) + mockedIterator + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkSuite.scala b/src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkTest.scala similarity index 92% rename from src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkSuite.scala rename to src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkTest.scala index 7e13ec66..e439209b 100644 --- a/src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/sink/BatchSizedSinkTest.scala @@ -2,7 +2,7 @@ package com.exasol.cloudetl.sink import java.nio.file.Path -import com.exasol.cloudetl.TestUtils +import com.exasol.cloudetl.DummyRecordsTest import com.exasol.cloudetl.bucket.LocalBucket import com.exasol.cloudetl.data.ExaColumnInfo import com.exasol.cloudetl.data.Row @@ -11,8 +11,7 @@ import com.exasol.cloudetl.storage.StorageProperties import org.scalatest.BeforeAndAfterEach import org.scalatest.FunSuite -@SuppressWarnings(Array("org.wartremover.warts.Var")) -class BatchSizedSinkSuite extends FunSuite with BeforeAndAfterEach with TestUtils { +class BatchSizedSinkTest extends FunSuite with BeforeAndAfterEach with DummyRecordsTest { private var outputPath: Path = _ private val properties = Map("BUCKET_PATH" -> "a/path", "DATA_FORMAT" -> "avro") diff --git a/src/test/scala/com/exasol/cloudetl/source/AvroSourceSuite.scala b/src/test/scala/com/exasol/cloudetl/source/AvroSourceSuite.scala deleted file mode 100644 index 10d3ce9c..00000000 --- a/src/test/scala/com/exasol/cloudetl/source/AvroSourceSuite.scala +++ /dev/null @@ -1,41 +0,0 @@ -package com.exasol.cloudetl.source - -import java.nio.file.Path -import java.nio.file.Paths - -import com.exasol.cloudetl.util.FileSystemUtil - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite -import org.scalatest.Matchers - -@SuppressWarnings(Array("org.wartremover.warts.Var")) -class AvroSourceSuite extends FunSuite with BeforeAndAfterAll with Matchers { - - private var conf: Configuration = _ - private var fileSystem: FileSystem = _ - private var avroResourceFolder: Path = _ - - override final def beforeAll(): Unit = { - conf = new Configuration() - fileSystem = FileSystem.get(conf) - avroResourceFolder = Paths.get(getClass.getResource("/data/import/avro").toURI).toAbsolutePath - () - } - - test("reads the sales avro format files") { - val filePath = Paths.get(s"$avroResourceFolder/sales1*.avro") - val globbedFilePath = FileSystemUtil.globWithLocal(filePath, fileSystem) - val result = globbedFilePath.map { file => - val source = AvroSource(file, conf, fileSystem) - val cnt = source.stream().size - source.close() - cnt - }.sum - - assert(result === 1998) - } - -} diff --git a/src/test/scala/com/exasol/cloudetl/source/AvroSourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/AvroSourceTest.scala new file mode 100644 index 00000000..2c9f7f86 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/source/AvroSourceTest.scala @@ -0,0 +1,14 @@ +package com.exasol.cloudetl.source + +import java.nio.file.Paths + +class AvroSourceTest extends SourceTest { + + override val format: String = "avro" + + test("stream returns count of records from AVRO files") { + val filePath = Paths.get(s"$resourceDir/sales1*.avro") + assert(getRecordsCount(filePath) === 1998) + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/source/OrcSourceSuite.scala b/src/test/scala/com/exasol/cloudetl/source/OrcSourceSuite.scala deleted file mode 100644 index 8b939583..00000000 --- a/src/test/scala/com/exasol/cloudetl/source/OrcSourceSuite.scala +++ /dev/null @@ -1,54 +0,0 @@ -package com.exasol.cloudetl.source - -import java.nio.file.Path -import java.nio.file.Paths - -import com.exasol.cloudetl.util.FileSystemUtil - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite -import org.scalatest.Matchers - -@SuppressWarnings(Array("org.wartremover.warts.Var")) -class OrcSourceSuite extends FunSuite with BeforeAndAfterAll with Matchers { - - private var conf: Configuration = _ - private var fileSystem: FileSystem = _ - private var orcResourceFolder: Path = _ - - override final def beforeAll(): Unit = { - conf = new Configuration() - fileSystem = FileSystem.get(conf) - orcResourceFolder = Paths.get(getClass.getResource("/data/import/orc").toURI).toAbsolutePath - () - } - - test("reads a single orc format file") { - val filePath = Paths.get(s"$orcResourceFolder/sales*.orc") - val globbedFilePath = FileSystemUtil.globWithLocal(filePath, fileSystem) - val result = globbedFilePath.map { file => - val source = OrcSource(file, conf, fileSystem) - val cnt = source.stream().size - source.close() - cnt - }.sum - - assert(result === 1998) - } - - test("reads an employee data orc file") { - val filePath = Paths.get(s"$orcResourceFolder/employee*.orc") - val globbedFilePath = FileSystemUtil.globWithLocal(filePath, fileSystem) - val result = globbedFilePath.map { file => - val source = OrcSource(file, conf, fileSystem) - val cnt = source.stream().size - source.close() - cnt - }.sum - - assert(result === 438304) - } - -} diff --git a/src/test/scala/com/exasol/cloudetl/source/OrcSourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/OrcSourceTest.scala new file mode 100644 index 00000000..89ad8957 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/source/OrcSourceTest.scala @@ -0,0 +1,19 @@ +package com.exasol.cloudetl.source + +import java.nio.file.Paths + +class OrcSourceTest extends SourceTest { + + override val format: String = "orc" + + test("stream returns count of records from single ORC file") { + val filePath = Paths.get(s"$resourceDir/sales*.orc") + assert(getRecordsCount(filePath) === 1998) + } + + test("stream returns count of records from ORC files") { + val filePath = Paths.get(s"$resourceDir/employee*.orc") + assert(getRecordsCount(filePath) === 438304) + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/source/ParquetSourceSuite.scala b/src/test/scala/com/exasol/cloudetl/source/ParquetSourceSuite.scala deleted file mode 100644 index 0088a38d..00000000 --- a/src/test/scala/com/exasol/cloudetl/source/ParquetSourceSuite.scala +++ /dev/null @@ -1,86 +0,0 @@ -package com.exasol.cloudetl.source - -import java.nio.file.Path -import java.nio.file.Paths - -import com.exasol.cloudetl.util.FileSystemUtil - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.apache.parquet.schema.MessageTypeParser -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite -import org.scalatest.Matchers - -@SuppressWarnings(Array("org.wartremover.warts.Var")) -class ParquetSourceSuite extends FunSuite with BeforeAndAfterAll with Matchers { - - private var conf: Configuration = _ - private var fileSystem: FileSystem = _ - private var parquetResourceFolder: Path = _ - - override final def beforeAll(): Unit = { - conf = new Configuration() - fileSystem = FileSystem.get(conf) - parquetResourceFolder = - Paths.get(getClass.getResource("/data/import/parquet/").toURI).toAbsolutePath - () - } - - test("reads a single sales_positions parquet format file") { - val filePath = Paths.get(s"$parquetResourceFolder/sales_positions1.snappy.parquet") - FileSystemUtil.globWithLocal(filePath, fileSystem).foreach { file => - val source = ParquetSource(file, conf, fileSystem) - assert(source.stream().size === 500) - } - } - - test("reads multiple sales_positions parquet format files") { - val filePattern = Paths.get(s"$parquetResourceFolder/sales_positions*.parquet") - val globbedFilePath = FileSystemUtil.globWithLocal(filePattern, fileSystem) - val result = globbedFilePath.map { file => - val source = ParquetSource(file, conf, fileSystem) - val cnt = source.stream().size - source.close() - cnt - }.sum - - assert(result === 1005) - } - - test("reads sales_positions parquet format files schema") { - val expectedMsgType = MessageTypeParser - .parseMessageType("""message spark_schema { - | optional int64 sales_id; - | optional int32 position_id; - | optional int32 article_id; - | optional int32 amount; - | optional double price; - | optional int32 voucher_id; - | optional boolean canceled; - |} - """.stripMargin) - - val filePattern = Paths.get(s"$parquetResourceFolder/sales_pos*.parquet") - val globbedFilePath = FileSystemUtil.globWithLocal(filePattern, fileSystem) - globbedFilePath.foreach { file => - val schema = ParquetSource(file, conf, fileSystem).getSchema() - assert(schema.isDefined) - schema.foreach { case msgType => assert(msgType === expectedMsgType) } - } - } - - test("reads a sales parquet format file with richer types") { - val filePath = Paths.get(s"$parquetResourceFolder/sales1.snappy.parquet") - val globbedFilePath = FileSystemUtil.globWithLocal(filePath, fileSystem) - val result = globbedFilePath.map { file => - val source = ParquetSource(file, conf, fileSystem) - val cnt = source.stream().size - source.close() - cnt - }.sum - - assert(result === 999) - } - -} diff --git a/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala new file mode 100644 index 00000000..db3be937 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala @@ -0,0 +1,50 @@ +package com.exasol.cloudetl.source + +import java.nio.file.Paths + +import com.exasol.cloudetl.util.FileSystemUtil + +import org.apache.parquet.schema.MessageTypeParser + +class ParquetSourceTest extends SourceTest { + + override val format: String = "parquet" + + test("getSchema returns parquet schema") { + val expectedMessageType = MessageTypeParser + .parseMessageType("""message spark_schema { + | optional int64 sales_id; + | optional int32 position_id; + | optional int32 article_id; + | optional int32 amount; + | optional double price; + | optional int32 voucher_id; + | optional boolean canceled; + |} + """.stripMargin) + + val filePattern = Paths.get(s"$resourceDir/sales_pos*.parquet") + val globbedFilePath = FileSystemUtil.globWithLocal(filePattern, getFileSystem()) + globbedFilePath.foreach { file => + val schema = ParquetSource(file, getConf(), getFileSystem()).getSchema() + assert(schema.isDefined) + schema.foreach { case messageType => assert(messageType === expectedMessageType) } + } + } + + test("stream returns count of records from single PARQUET file") { + val filePath = Paths.get(s"$resourceDir/sales_positions1.snappy.parquet") + assert(getRecordsCount(filePath) === 500) + } + + test("stream reaturns count of records from PARQUET files") { + val filePattern = Paths.get(s"$resourceDir/sales_positions*.parquet") + assert(getRecordsCount(filePattern) === 1005) + } + + test("stream returns count of records from PARQUET files with richer types") { + val filePath = Paths.get(s"$resourceDir/sales1.snappy.parquet") + assert(getRecordsCount(filePath) === 999) + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/source/SourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/SourceTest.scala new file mode 100644 index 00000000..8848dabd --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/source/SourceTest.scala @@ -0,0 +1,43 @@ +package com.exasol.cloudetl.source + +import java.nio.file.Path +import java.nio.file.Paths + +import com.exasol.cloudetl.storage.FileFormat +import com.exasol.cloudetl.util.FileSystemUtil + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.scalatest.BeforeAndAfterEach +import org.scalatest.FunSuite + +class SourceTest extends FunSuite with BeforeAndAfterEach { + + private[this] var conf: Configuration = _ + private[this] var fileSystem: FileSystem = _ + + private[source] val format: String = "dummy" + private[source] var resourceDir: Path = _ + + override final def beforeEach(): Unit = { + conf = new Configuration() + fileSystem = FileSystem.get(conf) + resourceDir = Paths.get(getClass.getResource(s"/data/import/$format").toURI).toAbsolutePath + () + } + + final def getConf(): Configuration = conf + + final def getFileSystem(): FileSystem = fileSystem + + final def getRecordsCount(filePath: Path): Int = { + val globbedFilePath = FileSystemUtil.globWithLocal(filePath, fileSystem) + globbedFilePath.map { file => + val source = Source(FileFormat(format), file, conf, fileSystem) + val cnt = source.stream().size + source.close() + cnt + }.sum + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/storage/FileFormatTest.scala b/src/test/scala/com/exasol/cloudetl/storage/FileFormatTest.scala index 25e71e78..6655e60f 100644 --- a/src/test/scala/com/exasol/cloudetl/storage/FileFormatTest.scala +++ b/src/test/scala/com/exasol/cloudetl/storage/FileFormatTest.scala @@ -1,5 +1,7 @@ package com.exasol.cloudetl.storage +import com.exasol.cloudetl.storage.FileFormat._ + import org.scalatest.FunSuite class FileFormatTest extends FunSuite { diff --git a/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala index f14c72e8..d9103d47 100644 --- a/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala +++ b/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala @@ -3,7 +3,6 @@ package com.exasol.cloudetl.storage import org.scalatest.BeforeAndAfterEach import org.scalatest.FunSuite -@SuppressWarnings(Array("org.wartremover.warts.Var")) class StoragePropertiesTest extends FunSuite with BeforeAndAfterEach { private[this] var properties: Map[String, String] = _ @@ -43,7 +42,7 @@ class StoragePropertiesTest extends FunSuite with BeforeAndAfterEach { StorageProperties.BUCKET_PATH -> "path", StorageProperties.DATA_FORMAT -> "orc" ) - assert(BaseProperties(properties).getFileFormat() === ORC) + assert(BaseProperties(properties).getFileFormat() === FileFormat.ORC) } test("getFileFormat throws if file format is not supported") { diff --git a/src/test/scala/com/exasol/cloudetl/util/DateTimeUtilSuite.scala b/src/test/scala/com/exasol/cloudetl/util/DateTimeUtilTest.scala similarity index 97% rename from src/test/scala/com/exasol/cloudetl/util/DateTimeUtilSuite.scala rename to src/test/scala/com/exasol/cloudetl/util/DateTimeUtilTest.scala index bc6e65db..de52d97e 100644 --- a/src/test/scala/com/exasol/cloudetl/util/DateTimeUtilSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/util/DateTimeUtilTest.scala @@ -8,7 +8,7 @@ import org.scalatest.FunSuite import org.scalatest.Matchers @SuppressWarnings(Array("org.wartremover.contrib.warts.OldTime")) -class DateTimeUtilSuite extends FunSuite with Matchers { +class DateTimeUtilTest extends FunSuite with Matchers { final def daysSinceEpochToDate(dt: Date): Unit = { val newDT = DateTimeUtil.daysToDate(DateTimeUtil.daysSinceEpoch(dt)) diff --git a/src/test/scala/com/exasol/cloudetl/util/FileSystemUtilSuite.scala b/src/test/scala/com/exasol/cloudetl/util/FileSystemUtilTest.scala similarity index 68% rename from src/test/scala/com/exasol/cloudetl/util/FileSystemUtilSuite.scala rename to src/test/scala/com/exasol/cloudetl/util/FileSystemUtilTest.scala index f4474192..b8a3fb43 100644 --- a/src/test/scala/com/exasol/cloudetl/util/FileSystemUtilSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/util/FileSystemUtilTest.scala @@ -6,44 +6,42 @@ import java.nio.file.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path -import org.scalatest.BeforeAndAfterAll +import org.scalatest.BeforeAndAfterEach import org.scalatest.FunSuite -import org.scalatest.Matchers -@SuppressWarnings(Array("org.wartremover.warts.Var")) -class FileSystemUtilSuite extends FunSuite with BeforeAndAfterAll with Matchers { +class FileSystemUtilTest extends FunSuite with BeforeAndAfterEach { - private var temporaryDirectory: FPath = _ - private var files: Seq[FPath] = _ + private[this] var temporaryDirectory: FPath = _ + private[this] var files: Seq[FPath] = _ - override final def beforeAll(): Unit = { + override final def beforeEach(): Unit = { temporaryDirectory = Files.createTempDirectory("tempdir") files = Seq("a", "b", "c", "a.parquet", "b.parquet").map(temporaryDirectory.resolve(_)) files.foreach { case f => Files.createFile(f) } () } - override final def afterAll(): Unit = { + override final def afterEach(): Unit = { files.foreach(Files.deleteIfExists) Files.deleteIfExists(temporaryDirectory) () } - test("`globWithPattern` method returns paths from a pattern") { + test("globWithPattern returns paths from a pattern") { val fs = FileSystem.get(new Configuration()) val expectedPaths = files.map(f => new Path(s"file:${f.toUri.getRawPath}")) val pathPattern = s"${temporaryDirectory.toUri.getRawPath}/*" assert(FileSystemUtil.globWithPattern(pathPattern, fs).toSet === expectedPaths.toSet) } - test("`globWithPattern` method returns paths from a pattern with file extensions") { + test("globWithPattern returns paths from a pattern with file extensions") { val fs = FileSystem.get(new Configuration()) val pathPattern = s"${temporaryDirectory.toUri.getRawPath}/*" val pathsWithExtensions = FileSystemUtil.globWithPattern(s"$pathPattern.parquet", fs) assert(pathsWithExtensions.map(_.toUri.getRawPath).forall(_.contains("parquet"))) } - test("`glob` method returns empty sequence if no path is available") { + test("glob returns empty sequence if no path exists") { val fileSystem = FileSystem.get(new Configuration()) assert(FileSystemUtil.glob(new Path("emptyPath"), fileSystem) === Seq.empty[Path]) } diff --git a/src/test/scala/com/exasol/cloudetl/util/SchemaUtilSuite.scala b/src/test/scala/com/exasol/cloudetl/util/SchemaUtilTest.scala similarity index 92% rename from src/test/scala/com/exasol/cloudetl/util/SchemaUtilSuite.scala rename to src/test/scala/com/exasol/cloudetl/util/SchemaUtilTest.scala index ae861944..bc6f9cda 100644 --- a/src/test/scala/com/exasol/cloudetl/util/SchemaUtilSuite.scala +++ b/src/test/scala/com/exasol/cloudetl/util/SchemaUtilTest.scala @@ -8,13 +8,12 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.Type.Repetition import org.mockito.Mockito._ import org.scalatest.FunSuite -import org.scalatest.Matchers import org.scalatest.mockito.MockitoSugar @SuppressWarnings(Array("org.wartremover.contrib.warts.ExposedTuples")) -class SchemaUtilSuite extends FunSuite with Matchers with MockitoSugar { +class SchemaUtilTest extends FunSuite with MockitoSugar { - test("`createParquetMessageType` throws an exception for unknown type") { + test("createParquetMessageType throws if type is unknown") { val thrown = intercept[IllegalArgumentException] { SchemaUtil.createParquetMessageType( Seq(ExaColumnInfo("c_short", classOf[java.lang.Short], 0, 0, 0, false)), @@ -25,8 +24,7 @@ class SchemaUtilSuite extends FunSuite with Matchers with MockitoSugar { assert(thrown.getMessage === expectedMsg) } - test("`createParquetMessageType` creates parquet message type from list of Exasol columns") { - + test("createParquetMessageType returns Parquet MessageType from Exasol columns") { val exasolColumns = Seq( ExaColumnInfo("c_int", classOf[java.lang.Integer], 0, 0, 0, true), ExaColumnInfo("c_int", classOf[java.lang.Integer], 1, 0, 0, true), @@ -116,9 +114,7 @@ class SchemaUtilSuite extends FunSuite with Matchers with MockitoSugar { assert(SchemaUtil.createParquetMessageType(exasolColumns, schemaName) === messageType) } - test( - "`createParquetMessageType` throws an exception if the integer precision is more than allowed" - ) { + test("createParquetMessageType throws if integer precision is larger than allowed") { val exasolColumns = Seq(ExaColumnInfo("c_int", classOf[java.lang.Integer], 10, 0, 0, true)) val thrown = intercept[IllegalArgumentException] { SchemaUtil.createParquetMessageType(exasolColumns, "test") @@ -127,9 +123,7 @@ class SchemaUtilSuite extends FunSuite with Matchers with MockitoSugar { assert(thrown.getMessage === expectedMsg) } - test( - "`createParquetMessageType` throws an exception if the long precision is more than allowed" - ) { + test("createParquetMessageType throws if long precision is larger than allowed") { val exasolColumns = Seq(ExaColumnInfo("c_long", classOf[java.lang.Long], 20, 0, 0, true)) val thrown = intercept[IllegalArgumentException] { SchemaUtil.createParquetMessageType(exasolColumns, "test") @@ -138,7 +132,7 @@ class SchemaUtilSuite extends FunSuite with Matchers with MockitoSugar { assert(thrown.getMessage === expectedMsg) } - test("`exaColumnToValue` returns value with column type") { + test("exaColumnToValue returns value with column type") { val iter = mock[ExaIterator] val startIdx = 3 val bd = new java.math.BigDecimal(1337) From dc5b115a1a3a4cdd3030a80acd3229b37b0a0682 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Tue, 15 Oct 2019 09:12:48 +0200 Subject: [PATCH 09/18] [ci skip] Add AUTHORS.md. Since patch send by Hari, is not visible in the Github contributors lists, give him credit with authors document. --- AUTHORS.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 AUTHORS.md diff --git a/AUTHORS.md b/AUTHORS.md new file mode 100644 index 00000000..f2b4abca --- /dev/null +++ b/AUTHORS.md @@ -0,0 +1,18 @@ +# Authors + +A successful open-source project depends on its [community of +contributors][contributors]. + +## Maintainers + +The maintainers of the project are: + +* Exasol Developers <[Exasol](https://github.com/exasol)> + +## Contributors + +These are the people whose contributions have made the project possible: + +* Hari Nair (CommScope) + +[contributors]: https://github.com/exasol/cloud-storage-etl-udfs/graphs/contributors From 42f1e057bc35e86f0be616daf044818fd730afa7 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Mon, 14 Oct 2019 16:27:05 +0200 Subject: [PATCH 10/18] Update Kafka properties handling. - Adds additional properties - Removes several old code and classes - KafkaConsumerProperties creates KafkaConsumer for scriptclasses --- .../cloudetl/kafka/KafkaConsumerBuilder.scala | 22 - .../kafka/KafkaConsumerProperties.scala | 570 +++++++++++++----- .../cloudetl/scriptclasses/KafkaImport.scala | 40 +- .../scriptclasses/KafkaMetadata.scala | 20 +- .../cloudetl/scriptclasses/KafkaPath.scala | 21 +- .../cloudetl/storage/StorageProperties.scala | 19 +- .../kafka/KafkaConsumerPropertiesTest.scala | 264 ++++++++ 7 files changed, 746 insertions(+), 210 deletions(-) delete mode 100644 src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerBuilder.scala create mode 100644 src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerBuilder.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerBuilder.scala deleted file mode 100644 index d8e9b901..00000000 --- a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerBuilder.scala +++ /dev/null @@ -1,22 +0,0 @@ -package com.exasol.cloudetl.kafka - -import org.apache.avro.generic.GenericRecord -import org.apache.kafka.clients.consumer.KafkaConsumer - -/** - * A companion object to the - * [[org.apache.kafka.clients.consumer.KafkaConsumer]] class. - */ -object KafkaConsumerBuilder { - - @SuppressWarnings(Array("org.wartremover.warts.Null")) - def apply(params: Map[String, String]): KafkaConsumer[String, GenericRecord] = { - val configs = KafkaConsumerProperties.fromImportParameters[GenericRecord](params) - new KafkaConsumer[String, GenericRecord]( - configs.getProperties(), - configs.keyDeserializerOpt.orNull, - configs.valueDeserializerOpt.orNull - ) - } - -} diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala index a5eb4db2..b713c7d7 100644 --- a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala @@ -3,60 +3,143 @@ package com.exasol.cloudetl.kafka import scala.collection.JavaConverters._ import scala.collection.mutable.{Map => MMap} -import com.exasol.cloudetl.bucket.Bucket +import com.exasol.cloudetl.common.AbstractProperties import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig import io.confluent.kafka.serializers.KafkaAvroDeserializer +import org.apache.avro.generic.GenericRecord import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.common.config.SslConfigs import org.apache.kafka.common.serialization.Deserializer import org.apache.kafka.common.serialization.StringDeserializer /** - * A companion object to the [[KafkaConsumerProperties]] class. + * A specific implementation of + * [[com.exasol.cloudetl.common.AbstractProperties]] that handles user + * provided key-value parameters for import user-defined-functions + * (udfs) as Kafka consumer application. * - * It provides helper functions such as {@code apply} and {@code create} - * for convenient properties construction. + * This class also provides builder methods for Kafka consumers. */ -@SuppressWarnings(Array("org.wartremover.warts.Overloading", "org.wartremover.warts.Null")) -object KafkaConsumerProperties { +class KafkaConsumerProperties(private val properties: Map[String, String]) + extends AbstractProperties(properties) { + + import KafkaConsumerProperties._ + + final def getBootstrapServers(): String = + getAs[String](BOOTSTRAP_SERVERS.userPropertyName) + + final def getGroupId(): String = + get(GROUP_ID.userPropertyName).fold(GROUP_ID.defaultValue)(identity) + + /** Returns the user provided topic name. */ + final def getTopics(): String = + getAs[String](TOPICS) + + /** Returns the user provided Exasol table name. */ + final def getTableName(): String = + getAs[String](TABLE_NAME) + + final def getPollTimeoutMs(): Long = + get(POLL_TIMEOUT_MS).fold(POLL_TIMEOUT_MS_DEFAULT_VALUE)(_.asInstanceOf[Long]) + + final def getMaxRecordsPerRun(): Int = + get(MAX_RECORDS_PER_RUN).fold(MAX_RECORDS_PER_RUN_DEFAULT_VALUE)(_.asInstanceOf[Int]) + + final def getMinRecordsPerRun(): Int = + get(MIN_RECORDS_PER_RUN).fold(MIN_RECORDS_PER_RUN_DEFAULT_VALUE)(_.asInstanceOf[Int]) + + /** Checks if the {@code SSL_ENABLED} property is set. */ + final def isSSLEnabled(): Boolean = + isEnabled(SSL_ENABLED) + + /** Checks if the Schema Registry URL property is set. */ + final def hasSchemaRegistryUrl(): Boolean = + containsKey(SCHEMA_REGISTRY_URL.userPropertyName) + + /** Returns the user provided schema registry url property. */ + final def getSchemaRegistryUrl(): String = + getAs[String](SCHEMA_REGISTRY_URL.userPropertyName) /** - * Creates Kafka consumer properties with optional key and value - * deserializers. + * Returns {@code MAX_POLL_RECORDS} property value if provided, + * otherwise returns default value. */ - def apply[K, V]( - properties: Map[String, String], - keyDeserializer: Option[Deserializer[K]], - valueDeserializer: Option[Deserializer[V]] - ): KafkaConsumerProperties[K, V] = { - require( - keyDeserializer != null && - (keyDeserializer.isDefined || properties - .contains(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG)), - "Key deserializer should be defined or specified in properties!" - ) - require( - valueDeserializer != null && - (valueDeserializer.isDefined || properties - .contains(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG)), - "Value deserializer should be defined or specified in properties!" - ) - new KafkaConsumerProperties[K, V](properties, keyDeserializer, valueDeserializer) - } + final def getMaxPollRecords(): String = + get(MAX_POLL_RECORDS.userPropertyName).fold(MAX_POLL_RECORDS.defaultValue)(identity) + + /** + * Returns {@code FETCH_MIN_BYTES} property value if provided, + * otherwise returns the default value. + */ + final def getFetchMinBytes(): String = + get(FETCH_MIN_BYTES.userPropertyName).fold(FETCH_MIN_BYTES.defaultValue)(identity) + + // Secure Connection Related Properties + + /** + * Returns {@code SECURITY_PROTOCOL} property value if provided, + * otherwise returns the default value. + */ + final def getSecurityProtocol(): String = + get(SECURITY_PROTOCOL.userPropertyName).fold(SECURITY_PROTOCOL.defaultValue)(identity) + + /** + * Returns the user provided {@code SSL_KEY_PASSWORD} property value. + */ + final def getSSLKeyPassword(): String = + getAs[String](SSL_KEY_PASSWORD.userPropertyName) + + /** + * Returns the user provided {@code SSL_KEYSTORE_PASSWORD} property + * value. + */ + final def getSSLKeystorePassword(): String = + getAs[String](SSL_KEYSTORE_PASSWORD.userPropertyName) + + /** + * Returns the user provided {@code SSL_KEYSTORE_LOCATION} property + * value. + */ + final def getSSLKeystoreLocation(): String = + getAs[String](SSL_KEYSTORE_LOCATION.userPropertyName) + + /** + * Returns the user provided {@code SSL_TRUSTSTORE_PASSWORD} property + * value. + */ + final def getSSLTruststorePassword(): String = + getAs[String](SSL_TRUSTSTORE_PASSWORD.userPropertyName) + + /** + * Returns the user provided {@code SSL_TRUSTSTORE_LOCATION} property + * value. + */ + final def getSSLTruststoreLocation(): String = + getAs[String](SSL_TRUSTSTORE_LOCATION.userPropertyName) + + /** + * Returns {@code SSL_ENDPOINT_IDENTIFICATION_ALGORITHM} property + * value if provided, otherwise returns the default value. + */ + final def getSSLEndpointIdentificationAlgorithm(): String = + get(SSL_ENDPOINT_IDENTIFICATION_ALGORITHM.userPropertyName) + .fold(SSL_ENDPOINT_IDENTIFICATION_ALGORITHM.defaultValue)(identity) /** - * Creates Kafka consumer properties with explicitly provided key and - * value deserializers. + * Returns a [[org.apache.kafka.clients.consumer.KafkaConsumer]] class. + * + * At the moment Avro based specific {@code KafkaConsumer[String, + * GenericRecord]} consumer is returned. Therefore, in order to define + * the schem of [[org.apache.avro.generic.GenericRecord]] the {@code + * SCHEMA_REGISTRY_URL} value should be provided. */ - def apply[K, V]( - properties: Map[String, String], - keyDeserializer: Deserializer[K], - valueDeserializer: Deserializer[V] - ): KafkaConsumerProperties[K, V] = - apply(properties, Option(keyDeserializer), Option(valueDeserializer)) + final def build(): KafkaConsumer[String, GenericRecord] = + KafkaConsumerProperties.createKafkaConsumer(this) + /** Returns the Kafka consumer properties as Java map. */ @SuppressWarnings( Array( "org.wartremover.warts.AsInstanceOf", @@ -64,135 +147,342 @@ object KafkaConsumerProperties { "org.wartremover.warts.NonUnitStatements" ) ) - def fromImportParameters[V]( - importParams: Map[String, String] - ): KafkaConsumerProperties[String, V] = { - val params = MMap.empty[String, String] - params.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") - params.put( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, - Bucket.requiredParam(importParams, "BROKER_ADDRESS") - ) - params.put(ConsumerConfig.GROUP_ID_CONFIG, Bucket.requiredParam(importParams, "GROUP_ID")) - val schemaRegistryUrl = Bucket.requiredParam(importParams, "SCHEMA_REGISTRY_URL") - params.put(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl) - params.put( - ConsumerConfig.MAX_POLL_RECORDS_CONFIG, - Bucket.optionalParameter(importParams, "MAX_POLL_RECORDS", "500") - ) - params.put( - ConsumerConfig.FETCH_MIN_BYTES_CONFIG, - Bucket.optionalParameter(importParams, "FETCH_MIN_BYTES", "1") - ) - val sslEnabled = Bucket.optionalParameter(importParams, "SSL_ENABLED", "false") - if (sslEnabled.equals("true")) { - params.put( - CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, - Bucket.requiredParam(importParams, "SECURITY_PROTOCOL") - ) - params.put( - SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, - Bucket.requiredParam(importParams, "SSL_KEYSTORE_LOCATION") - ) - params.put( - SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, - Bucket.requiredParam(importParams, "SSL_KEYSTORE_PASSWORD") - ) - params.put( - SslConfigs.SSL_KEY_PASSWORD_CONFIG, - Bucket.requiredParam(importParams, "SSL_KEY_PASSWORD") - ) - params.put( - SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, - Bucket.requiredParam(importParams, "SSL_TRUSTSTORE_LOCATION") - ) - params.put( - SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, - Bucket.requiredParam(importParams, "SSL_TRUSTSTORE_PASSWORD") - ) - val idAlgo = Bucket.optionalParameter( - importParams, - "SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", - SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM - ) - params.put( - SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, - if (idAlgo == "none") "" else idAlgo + final def getProperties(): java.util.Map[String, AnyRef] = { + val props = MMap.empty[String, String] + props.put(ENABLE_AUTO_COMMIT.kafkaPropertyName, ENABLE_AUTO_COMMIT.defaultValue) + props.put(BOOTSTRAP_SERVERS.kafkaPropertyName, getBootstrapServers()) + props.put(GROUP_ID.kafkaPropertyName, getGroupId()) + props.put(SCHEMA_REGISTRY_URL.kafkaPropertyName, getSchemaRegistryUrl()) + props.put(MAX_POLL_RECORDS.kafkaPropertyName, getMaxPollRecords()) + props.put(FETCH_MIN_BYTES.kafkaPropertyName, getFetchMinBytes()) + if (isSSLEnabled()) { + props.put(SECURITY_PROTOCOL.kafkaPropertyName, getSecurityProtocol()) + props.put(SSL_KEY_PASSWORD.kafkaPropertyName, getSSLKeyPassword()) + props.put(SSL_KEYSTORE_PASSWORD.kafkaPropertyName, getSSLKeystorePassword()) + props.put(SSL_KEYSTORE_LOCATION.kafkaPropertyName, getSSLKeystoreLocation()) + props.put(SSL_TRUSTSTORE_PASSWORD.kafkaPropertyName, getSSLTruststorePassword()) + props.put(SSL_TRUSTSTORE_LOCATION.kafkaPropertyName, getSSLTruststoreLocation()) + props.put( + SSL_ENDPOINT_IDENTIFICATION_ALGORITHM.kafkaPropertyName, + getSSLEndpointIdentificationAlgorithm() ) } - - KafkaConsumerProperties( - params.toMap, - new StringDeserializer, - createAvroDeserializer(schemaRegistryUrl).asInstanceOf[Deserializer[V]] - ) - + props.toMap.asInstanceOf[Map[String, AnyRef]].asJava } - def createAvroDeserializer(schemaRegistryUrl: String): KafkaAvroDeserializer = { - // The schema registry url should be provided here since the one - // configured in consumer properties is not for the deserializer. - val deserializerConfig = Map( - AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG -> schemaRegistryUrl - ) - val kafkaAvroDeserializer = new KafkaAvroDeserializer - kafkaAvroDeserializer.configure(deserializerConfig.asJava, false) - kafkaAvroDeserializer - } + final def getAs[T](key: String): T = + get(key).fold { + throw new IllegalArgumentException(s"Please provide a value for the $key property!") + }(_.asInstanceOf[T]) + + /** + * Returns a string value of key-value property pairs. + * + * The resulting string is sorted by keys ordering. + */ + @SuppressWarnings(Array("org.wartremover.warts.Overloading")) + final def mkString(): String = + mkString(KEY_VALUE_SEPARATOR, PROPERTY_SEPARATOR) } /** - * A properties holder class for Kafka consumers. - * - * It is parameterized on key/value deserializer types. + * A companion object for [[KafkaConsumerProperties]] class. */ -class KafkaConsumerProperties[K, V]( - val properties: Map[String, String], - val keyDeserializerOpt: Option[Deserializer[K]], - val valueDeserializerOpt: Option[Deserializer[V]] -) { +object KafkaConsumerProperties { + + /** + * A line separator string used for creating key-value property + * strings. + */ + private[kafka] final val PROPERTY_SEPARATOR: String = ";" + + /** + * A default separator string used for concatenate key-value pairs. + */ + private[kafka] final val KEY_VALUE_SEPARATOR: String = " -> " + + /** + * A required property key name for a Kafka topic name to import data + * from. + */ + private[kafka] final val TOPICS: String = "TOPICS" + + /** + * A required property key name for a Exasol table name to import data + * into. + */ + private[kafka] final val TABLE_NAME: String = "TABLE_NAME" + + private[kafka] final val POLL_TIMEOUT_MS: String = "POLL_TIMEOUT_MS" + private[kafka] final val POLL_TIMEOUT_MS_DEFAULT_VALUE: Long = 30000L + + private[kafka] final val MAX_RECORDS_PER_RUN: String = "MAX_RECORDS_PER_RUN" + private[kafka] final val MAX_RECORDS_PER_RUN_DEFAULT_VALUE: Int = 1000000 + + private[kafka] final val MIN_RECORDS_PER_RUN: String = "MIN_RECORDS_PER_RUN" + private[kafka] final val MIN_RECORDS_PER_RUN_DEFAULT_VALUE: Int = 100 + + /** + * An optional property key name to set SSL secure connections to + * Kafka cluster. + */ + private[kafka] val SSL_ENABLED: String = "SSL_ENABLED" + + /** + * Below are relavant Kafka consumer configuration parameters are + * defined. + * + * See [[https://kafka.apache.org/documentation.html#consumerconfigs]] + */ + /** + * Internal configuration helper class. + * + * @param userPropertyName A UDF user provided property key name + * @param kafkaPropertyName An equivalent property in Kafka + * configuration that maps user property key name + * @param defaultValue A default value for the property key name + */ + private[kafka] final case class Config( + val userPropertyName: String, + val kafkaPropertyName: String, + val defaultValue: String + ) + + /** + * This is the {@code enable.auto.commit} configuration setting. + * + * If set to true the offset of consumer will be periodically + * committed to the Kafka cluster in the background. This is `false` + * by default, since we manage the offset commits ourselves in the + * Exasol table. + */ + private[kafka] val ENABLE_AUTO_COMMIT: Config = Config( + "ENABLE_AUTO_COMMIT", + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, + "false" + ) + + /** + * This is the `bootstrap.servers` configuration setting. + * + * A list of host and port pairs to use for establishing the initial + * connection to the Kafka cluster. + * + * It is a required property that should be provided by the user. + */ + private[kafka] val BOOTSTRAP_SERVERS: Config = Config( + "BOOTSTRAP_SERVERS", + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, + "" + ) + + /** + * This is the {@code group.id} configuration setting. + * + * It is a unique string that identifies the consumer group this + * consumer belongs to. + */ + private[kafka] val GROUP_ID: Config = Config( + "GROUP_ID", + ConsumerConfig.GROUP_ID_CONFIG, + "EXASOL_KAFKA_UDFS_CONSUMERS" + ) + + /** + * This is the {@code max.poll.records} configuration setting. + * + * It is the maximum number of records returned in a single call to + * poll() function. Default value is `500`. + */ + private[kafka] val MAX_POLL_RECORDS: Config = Config( + "MAX_POLL_RECORDS", + ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + "500" + ) + + /** + * This is the {@code fetch.min.bytes} configuration setting. + * + * It is the minimum amount of data the server should return for a + * fetch request. Default value is `1`. + */ + private[kafka] val FETCH_MIN_BYTES: Config = Config( + "FETCH_MIN_BYTES", + ConsumerConfig.FETCH_MIN_BYTES_CONFIG, + "1" + ) + + /** + * An optional schema registry url. + * + * The Avro value deserializer will be used when user sets this + * property value. + */ + private[kafka] val SCHEMA_REGISTRY_URL: Config = Config( + "SCHEMA_REGISTRY_URL", + AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + "" + ) + + /** + * This is the {@code security.protocol} configuration setting. + * + * It is the protocol used to communicate with brokers, when + * [[SSL_ENABLED]] is set to {@code true}. Default value is + * [[SslConfigs.DEFAULT_SSL_PROTOCOL]]. + */ + private[kafka] val SECURITY_PROTOCOL: Config = Config( + "SECURITY_PROTOCOL", + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, + SslConfigs.DEFAULT_SSL_PROTOCOL + ) /** - * A comma-separated collection of host/port pairs in order to connect - * to Kafka brokers. + * This is the {@code ssl.key.password} configuration setting. + * + * It represents the password of the private key in the key store + * file. It is required property when [[SSL_ENABLED]] is set to {@code + * true}. */ - final def withBootstrapServers(bootstrapServers: String): KafkaConsumerProperties[K, V] = - withProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers) + private[kafka] val SSL_KEY_PASSWORD: Config = Config( + "SSL_KEY_PASSWORD", + SslConfigs.SSL_KEY_PASSWORD_CONFIG, + "" + ) + + /** + * This is the {@code ssl.keystore.password} confguration setting. + * + * It the store password for the keystore file. It is required + * property when [[SSL_ENABLED]] is set to {@code true}. + */ + private[kafka] val SSL_KEYSTORE_PASSWORD: Config = Config( + "SSL_KEYSTORE_PASSWORD", + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, + "" + ) + + /** + * This is the {@code ssl.keystore.location} configuration setting. + * + * It represents the location of the keystore file. It is required + * property when [[SSL_ENABLED]] is set to {@code true} and can be + * used for two-way authentication for the clients. + */ + private[kafka] val SSL_KEYSTORE_LOCATION: Config = Config( + "SSL_KEYSTORE_LOCATION", + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, + "" + ) + + /** + * This is the {@code ssl.truststore.password} configuration setting. + * + * It is the password for the truststore file, and required property + * when [[SSL_ENABLED]] is set to {@code true}. + */ + private[kafka] val SSL_TRUSTSTORE_PASSWORD: Config = Config( + "SSL_TRUSTSTORE_PASSWORD", + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, + "" + ) + + /** + * This is the {@code ssl.truststore.location} configuration setting. + * + * It is the location of the truststore file, and required property + * when [[SSL_ENABLED]] is set to {@code true}. + */ + private[kafka] val SSL_TRUSTSTORE_LOCATION: Config = Config( + "SSL_TRUSTSTORE_LOCATION", + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, + "" + ) /** - * A unique identifier the consumer group this consumer belongs. + * This is the {@code ssl.endpoint.identification.algorithm} + * configuration setting. + * + * It is the endpoint identification algorithm to validate server + * hostname using server certificate. It is used when [[SSL_ENABLED]] + * is set to {@code true}. Default value is + * [[SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM]]. */ - final def withGroupId(groupId: String): KafkaConsumerProperties[K, V] = - withProperty(ConsumerConfig.GROUP_ID_CONFIG, groupId) + private[kafka] val SSL_ENDPOINT_IDENTIFICATION_ALGORITHM: Config = Config( + "SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", + SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, + SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM + ) /** - * A schema registry url this consumer can use. + * Returns [[KafkaConsumerProperties]] from user provided key value + * properties. */ - final def withSchemaRegistryUrl(schemaRegistryUrl: String): KafkaConsumerProperties[K, V] = - withProperty(AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistryUrl) + def apply(params: Map[String, String]): KafkaConsumerProperties = + new KafkaConsumerProperties(params) /** - * Sets or updates key/value Kafka consumer property. + * Creates [[KafkaConsumerProperties]] from properly separated string. */ - final def withProperty(key: String, value: String): KafkaConsumerProperties[K, V] = - copy(properties = properties.updated(key, value)) + def fromString(string: String): KafkaConsumerProperties = { + if (!string.contains(PROPERTY_SEPARATOR)) { + throw new IllegalArgumentException( + s"The input string is not separated by '$PROPERTY_SEPARATOR'!" + ) + } + val properties = string + .split(PROPERTY_SEPARATOR) + .map { word => + val pairs = word.split(KEY_VALUE_SEPARATOR) + pairs(0) -> pairs(1) + } + .toMap + + new KafkaConsumerProperties(properties) + } /** - * Returns the Kafka consumer properties as Java map. + * Creates a [[org.apache.kafka.clients.consumer.KafkaConsumer]] from + * [[KafkaConsumerProperties]]. */ - final def getProperties(): java.util.Map[String, AnyRef] = - properties.asInstanceOf[Map[String, AnyRef]].asJava + @SuppressWarnings(Array("org.wartremover.warts.AsInstanceOf")) + def createKafkaConsumer( + properties: KafkaConsumerProperties + ): KafkaConsumer[String, GenericRecord] = { + validate(properties) + new KafkaConsumer[String, GenericRecord]( + properties.getProperties(), + new StringDeserializer, + getAvroDeserializer(properties.getSchemaRegistryUrl()) + .asInstanceOf[Deserializer[GenericRecord]] + ) + } + + private[this] def validate(properties: KafkaConsumerProperties): Unit = { + if (!properties.containsKey(BOOTSTRAP_SERVERS.userPropertyName)) { + throw new IllegalArgumentException( + s"Please provide a value for the " + + s"${BOOTSTRAP_SERVERS.userPropertyName} property!" + ) + } + if (!properties.hasSchemaRegistryUrl()) { + throw new IllegalArgumentException( + s"Please provide a value for the " + + s"${SCHEMA_REGISTRY_URL.userPropertyName} property!" + ) + } + } - @SuppressWarnings(Array("org.wartremover.warts.DefaultArguments")) - private[this] def copy( - properties: Map[String, String], - keyDeserializer: Option[Deserializer[K]] = keyDeserializerOpt, - valueDeserializer: Option[Deserializer[V]] = valueDeserializerOpt - ): KafkaConsumerProperties[K, V] = - new KafkaConsumerProperties[K, V]( - properties, - keyDeserializer, - valueDeserializer + private[this] def getAvroDeserializer(schemaRegistryUrl: String): KafkaAvroDeserializer = { + // The schema registry url should be provided here since the one + // configured in consumer properties is not for the deserializer. + val deserializerConfig = Map( + AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG -> schemaRegistryUrl ) + val kafkaAvroDeserializer = new KafkaAvroDeserializer + kafkaAvroDeserializer.configure(deserializerConfig.asJava, false) + kafkaAvroDeserializer + } + } diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala index e30954de..04df3f66 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala @@ -7,44 +7,34 @@ import scala.collection.JavaConverters._ import com.exasol.ExaIterator import com.exasol.ExaMetadata -import com.exasol.cloudetl.bucket.Bucket import com.exasol.cloudetl.data.Row -import com.exasol.cloudetl.kafka.KafkaConsumerBuilder +import com.exasol.cloudetl.kafka.KafkaConsumerProperties import com.typesafe.scalalogging.LazyLogging import org.apache.kafka.common.TopicPartition object KafkaImport extends LazyLogging { - private[this] val POLL_TIMEOUT_MS: Int = 30000 - - private[this] val MAX_RECORDS_PER_RUN = 1000000 - - private[this] val MIN_RECORDS_PER_RUN = 100 - @SuppressWarnings(Array("org.wartremover.warts.AsInstanceOf")) - def run(meta: ExaMetadata, ctx: ExaIterator): Unit = { - val rest = ctx.getString(0) - val partitionId = ctx.getInteger(1) - val partitionOffset = ctx.getLong(2) + def run(metadata: ExaMetadata, iterator: ExaIterator): Unit = { + val kafkaProperties = KafkaConsumerProperties.fromString(iterator.getString(0)) + val partitionId = iterator.getInteger(1) + val partitionOffset = iterator.getLong(2) val partitionNextOffset = partitionOffset + 1L - val nodeId = meta.getNodeId - val vmId = meta.getVmId + val nodeId = metadata.getNodeId + val vmId = metadata.getVmId logger.info( s"Kafka consumer for node=$nodeId, vm=$vmId using " + s"partition=$partitionId and startOffset=$partitionNextOffset" ) - val params = Bucket.keyValueStringToMap(rest) - val topics = Bucket.requiredParam(params, "TOPICS") - val timeout = Bucket.optionalIntParameter(params, "POLL_TIMEOUT_MS", POLL_TIMEOUT_MS) - val maxRecords = - Bucket.optionalIntParameter(params, "MAX_RECORDS_PER_RUN", MAX_RECORDS_PER_RUN) - val minRecords = - Bucket.optionalIntParameter(params, "MIN_RECORDS_PER_RUN", MIN_RECORDS_PER_RUN) + val topics = kafkaProperties.getTopics() + val timeout = kafkaProperties.getPollTimeoutMs() + val maxRecords = kafkaProperties.getMaxRecordsPerRun() + val minRecords = kafkaProperties.getMinRecordsPerRun() val topicPartition = new TopicPartition(topics, partitionId) - val kafkaConsumer = KafkaConsumerBuilder(params) + val kafkaConsumer = kafkaProperties.build() kafkaConsumer.assign(Arrays.asList(topicPartition)) kafkaConsumer.seek(topicPartition, partitionNextOffset) @@ -56,7 +46,7 @@ object KafkaImport extends LazyLogging { var total = 0 do { - val records = kafkaConsumer.poll(Duration.ofMillis(timeout.toLong)) + val records = kafkaConsumer.poll(Duration.ofMillis(timeout)) recordsCount = records.count() total += recordsCount records.asScala.foreach { record => @@ -68,9 +58,7 @@ object KafkaImport extends LazyLogging { Seq(record.partition().asInstanceOf[AnyRef], record.offset().asInstanceOf[AnyRef]) val row = Row.fromAvroGenericRecord(record.value()) val allColumns: Seq[Object] = metadata ++ row.getValues().map(_.asInstanceOf[AnyRef]) - ctx.emit( - allColumns: _* - ) + iterator.emit(allColumns: _*) } logger.info( s"Emitted total=$recordsCount records in node=$nodeId, vm=$vmId, partition=$partitionId" diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala index dc7dd0e7..3f8840fb 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala @@ -9,26 +9,24 @@ import scala.util.{Failure, Success, Try} import com.exasol.ExaIterator import com.exasol.ExaMetadata -import com.exasol.cloudetl.bucket.Bucket -import com.exasol.cloudetl.kafka.KafkaConsumerBuilder +import com.exasol.cloudetl.kafka.KafkaConsumerProperties import com.typesafe.scalalogging.LazyLogging object KafkaMetadata extends LazyLogging { @SuppressWarnings(Array("org.wartremover.warts.MutableDataStructures")) - def run(meta: ExaMetadata, iter: ExaIterator): Unit = { - val rest = iter.getString(0) - val params = Bucket.keyValueStringToMap(rest) + def run(metadata: ExaMetadata, iterator: ExaIterator): Unit = { + val kafkaProperties = KafkaConsumerProperties.fromString(iterator.getString(0)) val idOffsetPairs: HashMap[JInt, JLong] = HashMap.empty[JInt, JLong] do { - val partitionId = iter.getInteger(1) - val partitionOffset = iter.getLong(2) + val partitionId = iterator.getInteger(1) + val partitionOffset = iterator.getLong(2) idOffsetPairs += (partitionId -> partitionOffset) - } while (iter.next()) + } while (iterator.next()) - val kafkaConsumerTry = Try(KafkaConsumerBuilder(params)) + val kafkaConsumerTry = Try(kafkaProperties.build()) kafkaConsumerTry match { case Failure(ex) => @@ -36,14 +34,14 @@ object KafkaMetadata extends LazyLogging { throw ex case Success(kafkaConsumer) => - val topics = Bucket.requiredParam(params, "TOPICS") + val topics = kafkaProperties.getTopics() val topicPartitions = kafkaConsumer.partitionsFor(topics).asScala.toList.map(_.partition()) try { topicPartitions.foreach { partitionId => val offset: JLong = idOffsetPairs.getOrElse(partitionId, -1) - iter.emit(new Integer(partitionId), offset) + iterator.emit(new Integer(partitionId), offset) } } finally { kafkaConsumer.close() diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala index 7c1bb6cb..5ba510d8 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaPath.scala @@ -4,27 +4,30 @@ import scala.collection.JavaConverters._ import com.exasol.ExaImportSpecification import com.exasol.ExaMetadata -import com.exasol.cloudetl.bucket.Bucket +import com.exasol.cloudetl.kafka.KafkaConsumerProperties object KafkaPath { - def generateSqlForImportSpec(exaMeta: ExaMetadata, exaSpec: ExaImportSpecification): String = { - val params = exaSpec.getParameters.asScala.toMap - val tableName = Bucket.requiredParam(params, "TABLE_NAME") - val topics = Bucket.requiredParam(params, "TOPICS") + def generateSqlForImportSpec( + metadata: ExaMetadata, + importSpec: ExaImportSpecification + ): String = { + val kafkaProperties = KafkaConsumerProperties(importSpec.getParameters.asScala.toMap) + val tableName = kafkaProperties.getTableName() + val topics = kafkaProperties.getTopics() if (topics.contains(",")) { throw new IllegalArgumentException("Only single topic can be consumed using Kafka import!") } - val rest = Bucket.keyValueMapToString(params) - val scriptSchema = exaMeta.getScriptSchema + val kvPairs = kafkaProperties.mkString() + val scriptSchema = metadata.getScriptSchema s"""SELECT | $scriptSchema.KAFKA_IMPORT( - | '$rest', partition_index, max_offset + | '$kvPairs', partition_index, max_offset |) |FROM ( | SELECT $scriptSchema.KAFKA_METADATA( - | '$rest', kafka_partition, kafka_offset + | '$kvPairs', kafka_partition, kafka_offset | ) FROM ( | SELECT kafka_partition, MAX(kafka_offset) AS kafka_offset | FROM $tableName diff --git a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala index 32fa0011..32bb1f8d 100644 --- a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala @@ -15,11 +15,21 @@ class StorageProperties(private val properties: Map[String, String]) import StorageProperties._ - /** Returns the storage main path. */ + /** + * Returns the storage path. + * + * It is a main path, for example, to a bucket where data files are + * stored. + */ final def getStoragePath(): String = getAs[String](BUCKET_PATH) - /** Returns the main storage path scheme. */ + /** + * Returns the storage path scheme. + * + * For example, given the S3 bucket, `s3a://my-bucket/data/` path to + * load data, returns the scheme `s3a` value. + */ final def getStoragePathScheme(): String = new URI(getStoragePath()).getScheme @@ -27,6 +37,11 @@ class StorageProperties(private val properties: Map[String, String]) final def getFileFormat(): FileFormat = FileFormat(getAs[String](DATA_FORMAT)) + /** + * Returns the number of partitions provided as user property. + * + * If it is not set, returns default value {@code nproc}. + */ final def getParallelism(defaultValue: => String): String = get(PARALLELISM).fold(defaultValue)(identity) diff --git a/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala new file mode 100644 index 00000000..0b140e2c --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala @@ -0,0 +1,264 @@ +package com.exasol.cloudetl.kafka + +import org.apache.avro.generic.GenericRecord +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.scalatest.BeforeAndAfterEach +import org.scalatest.FunSuite + +@SuppressWarnings(Array("org.wartremover.warts.IsInstanceOf")) +class KafkaConsumerPropertiesTest extends FunSuite with BeforeAndAfterEach { + + private[this] var properties: Map[String, String] = _ + + override final def beforeEach(): Unit = { + properties = Map.empty[String, String] + () + } + + private[this] def errorMessage(key: String): String = + s"Please provide a value for the $key property!" + + test("getBootstrapServers returns bootstrap servers property value") { + val bootstrapServers = "kafka01.example.com,kafka02.example.com" + properties = Map("BOOTSTRAP_SERVERS" -> bootstrapServers) + assert(BaseProperties(properties).getBootstrapServers() === bootstrapServers) + } + + test("getBootstrapServers throws if bootstrap servers property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getBootstrapServers() + } + assert(thrown.getMessage === errorMessage("BOOTSTRAP_SERVERS")) + } + + test("getGroupId returns user provided value") { + properties = Map("GROUP_ID" -> "groupId") + assert(BaseProperties(properties).getGroupId() === "groupId") + } + + test("getGroupId returns default value if group id is not set") { + assert(BaseProperties(properties).getGroupId() === "EXASOL_KAFKA_UDFS_CONSUMERS") + } + + test("getTopics returns topics property value") { + properties = Map("TOPICS" -> "Metamorphosis") + assert(BaseProperties(properties).getTopics() === "Metamorphosis") + } + + test("getTopics throws if topics property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getTopics() + } + assert(thrown.getMessage === errorMessage("TOPICS")) + } + + test("getTableName returns Exasol table name property value") { + properties = Map("TABLE_NAME" -> "table") + assert(BaseProperties(properties).getTableName() === "table") + } + + test("getTableName throws if table name property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getTableName() + } + assert(thrown.getMessage === errorMessage("TABLE_NAME")) + } + + test("getPollTimeoutMs returns provided poll timeout value") { + properties = Map("POLL_TIMEOUT_MS" -> "10") + assert(BaseProperties(properties).getPollTimeoutMs() === 10L) + } + + test("getPollTimeoutMs returns default value if poll timeout is not set") { + assert(BaseProperties(properties).getPollTimeoutMs() === 30000L) + } + + test("getPollTimeoutMs throws if value cannot be converted to long") { + properties = Map("POLL_TIMEOUT_MS" -> "1l") + intercept[NumberFormatException] { + BaseProperties(properties).getPollTimeoutMs() + } + } + + test("getMinRecordsPerRun returns provided minimum value") { + properties = Map("MIN_RECORDS_PER_RUN" -> "7") + assert(BaseProperties(properties).getMinRecordsPerRun() === 7) + } + + test("getMinRecordsPerRun returns default value if minimum value is not set") { + assert(BaseProperties(properties).getMinRecordsPerRun() === 100) + } + + test("getMinRecordsPerRun throws if value cannot be converted to int") { + properties = Map("MIN_RECORDS_PER_RUN" -> "e") + intercept[NumberFormatException] { + BaseProperties(properties).getMinRecordsPerRun() + } + } + + test("getMaxRecordsPerRun returns provided maximum value") { + properties = Map("MAX_RECORDS_PER_RUN" -> "43") + assert(BaseProperties(properties).getMaxRecordsPerRun() === 43) + } + + test("getMaxRecordsPerRun returns default value if maximum value is not set") { + assert(BaseProperties(properties).getMaxRecordsPerRun() === 1000000) + } + + test("getMaxRecordsPerRun throws if value cannot be converted to int") { + properties = Map("MAX_RECORDS_PER_RUN" -> "max") + intercept[NumberFormatException] { + BaseProperties(properties).getMaxRecordsPerRun() + } + } + + test("isSSLEnabled returns true if it is set to true") { + properties = Map("SSL_ENABLED" -> "true") + assert(BaseProperties(properties).isSSLEnabled() === true) + } + + test("isSSLEnabled returns false if it is not set") { + assert(BaseProperties(properties).isSSLEnabled() === false) + } + + test("hasSchemaRegistryUrl returns true if schema registry url is provided") { + properties = Map("SCHEMA_REGISTRY_URL" -> "https://schema-registry.example.com") + assert(BaseProperties(properties).hasSchemaRegistryUrl() === true) + } + + test("hasSchemaRegistryUrl returns false if schema registry url is not set") { + assert(BaseProperties(properties).hasSchemaRegistryUrl() === false) + } + + test("getSchemaRegistryUrl returns schema registry url property value") { + properties = Map("SCHEMA_REGISTRY_URL" -> "http://a-schema.url") + assert(BaseProperties(properties).getSchemaRegistryUrl() === "http://a-schema.url") + } + + test("getSchemaRegistryUrl throws if schema registry url property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getSchemaRegistryUrl() + } + assert(thrown.getMessage === errorMessage("SCHEMA_REGISTRY_URL")) + } + + test("getMaxPollRecords returns max poll records value") { + properties = Map("MAX_POLL_RECORDS" -> "9") + assert(BaseProperties(properties).getMaxPollRecords() === "9") + } + + test("getMaxPollRecords returns default value if max poll records is not set") { + assert(BaseProperties(properties).getMaxPollRecords() === "500") + } + + test("getFetchMinBytes returns minimum fetch bytes property value") { + properties = Map("FETCH_MIN_BYTES" -> "23") + assert(BaseProperties(properties).getFetchMinBytes() === "23") + } + + test("getFetchMinBytes returns default value if property is not set") { + assert(BaseProperties(properties).getFetchMinBytes() === "1") + } + + test("getSecurityProtocol returns user provided security protocol property value") { + properties = Map("SECURITY_PROTOCOL" -> "SSL") + assert(BaseProperties(properties).getSecurityProtocol() === "SSL") + } + + test("getSecurityProtocol returns default value if security protocol is not set") { + // default value is intentionally hardcoded, should alert if things + // change + assert(BaseProperties(properties).getSecurityProtocol() === "TLS") + } + + test("getSSLKeyPassword returns ssl key password property value") { + properties = Map("SSL_KEY_PASSWORD" -> "1337") + assert(BaseProperties(properties).getSSLKeyPassword() === "1337") + } + + test("getSSLKeyPassword throws if ssl key password property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getSSLKeyPassword() + } + assert(thrown.getMessage === errorMessage("SSL_KEY_PASSWORD")) + } + + test("getSSLKeystorePassword returns ssl keystore password property value") { + properties = Map("SSL_KEYSTORE_PASSWORD" -> "p@ss") + assert(BaseProperties(properties).getSSLKeystorePassword() === "p@ss") + } + + test("getSSLKeystorePassword throws if ssl keystore password property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getSSLKeystorePassword() + } + assert(thrown.getMessage === errorMessage("SSL_KEYSTORE_PASSWORD")) + } + + test("getSSLKeystoreLocation returns ssl keystore location property value") { + properties = Map("SSL_KEYSTORE_LOCATION" -> "/path/keystore.jks") + assert(BaseProperties(properties).getSSLKeystoreLocation() === "/path/keystore.jks") + } + + test("getSSLKeystoreLocation throws if ssl keystore location property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getSSLKeystoreLocation() + } + assert(thrown.getMessage === errorMessage("SSL_KEYSTORE_LOCATION")) + } + + test("getSSLTruststorePassword returns ssl truststore password property value") { + properties = Map("SSL_TRUSTSTORE_PASSWORD" -> "tp@ss") + assert(BaseProperties(properties).getSSLTruststorePassword() === "tp@ss") + } + + test("getSSLTruststorePassword throws if ssl truststore password property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getSSLTruststorePassword() + } + assert(thrown.getMessage === errorMessage("SSL_TRUSTSTORE_PASSWORD")) + } + + test("getSSLTruststoreLocation returns ssl truststore location property value") { + properties = Map("SSL_TRUSTSTORE_LOCATION" -> "/path/truststore.jks") + assert(BaseProperties(properties).getSSLTruststoreLocation() === "/path/truststore.jks") + } + + test("getSSLTruststorePassword throws if ssl truststore location property is not set") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getSSLTruststoreLocation() + } + assert(thrown.getMessage === errorMessage("SSL_TRUSTSTORE_LOCATION")) + } + + test("getSSLEndpointIdentificationAlgorithm returns user provided property value") { + properties = Map("SSL_ENDPOINT_IDENTIFICATION_ALGORITHM" -> "none") + assert(BaseProperties(properties).getSSLEndpointIdentificationAlgorithm() === "none") + } + + test("getSSLEndpointIdentificationAlgorithm returns default value if it is not set") { + // default value is intentionally hardcoded, should alert if things + // change + assert(BaseProperties(properties).getSSLEndpointIdentificationAlgorithm() === "https") + } + + test("build throws if required properties are not provided") { + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).build() + } + assert(thrown.getMessage === errorMessage("BOOTSTRAP_SERVERS")) + } + + ignore("build returns a KafkaConsumer[String, GenericRecord]") { + properties = Map( + "BOOTSTRAP_SERVERS" -> "kafka01.internal:9092", + "SCHEMA_REGISTRY_URL" -> "https://schema-registry.internal.com" + ) + val kafkaConsumer = BaseProperties(properties).build() + assert(kafkaConsumer.isInstanceOf[KafkaConsumer[String, GenericRecord]]) + } + + private[this] case class BaseProperties(val params: Map[String, String]) + extends KafkaConsumerProperties(params) + +} From ae1bd3cbada56f19f7e6ef0e3e089bb297c3aea4 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Tue, 15 Oct 2019 12:23:10 +0200 Subject: [PATCH 11/18] Separate properties companion objects common parts into its own trait. --- .../cloudetl/common/CommonProperties.scala | 39 +++++++++++++++++++ .../kafka/KafkaConsumerProperties.scala | 35 +++-------------- .../cloudetl/scriptclasses/ExportTable.scala | 2 +- .../cloudetl/scriptclasses/ImportFiles.scala | 2 +- .../scriptclasses/ImportMetadata.scala | 2 +- .../cloudetl/scriptclasses/KafkaImport.scala | 2 +- .../scriptclasses/KafkaMetadata.scala | 2 +- .../cloudetl/storage/StorageProperties.scala | 38 ++++-------------- .../common/CommonPropertiesTest.scala | 27 +++++++++++++ .../cloudetl/orc/OrcDeserializerTest.scala | 2 +- .../storage/StoragePropertiesTest.scala | 23 +++++++++-- 11 files changed, 104 insertions(+), 70 deletions(-) create mode 100644 src/main/scala/com/exasol/cloudetl/common/CommonProperties.scala create mode 100644 src/test/scala/com/exasol/cloudetl/common/CommonPropertiesTest.scala diff --git a/src/main/scala/com/exasol/cloudetl/common/CommonProperties.scala b/src/main/scala/com/exasol/cloudetl/common/CommonProperties.scala new file mode 100644 index 00000000..26f37bcc --- /dev/null +++ b/src/main/scala/com/exasol/cloudetl/common/CommonProperties.scala @@ -0,0 +1,39 @@ +package com.exasol.cloudetl.common + +/** + * A trait with common variables and methods that can be used in + * companion objects of specific properties classes. + */ +trait CommonProperties { + + /** + * A line separator string used for creating key-value property + * strings. + */ + final val PROPERTY_SEPARATOR: String = ";" + + /** + * A default separator string used for concatenate key-value pairs. + */ + final val KEY_VALUE_SEPARATOR: String = " -> " + + /** + * Parses propertly separated input string into key value pair map + * data structure. + */ + final def mapFromString(string: String): Map[String, String] = { + if (!string.contains(PROPERTY_SEPARATOR)) { + throw new IllegalArgumentException( + s"The input string is not separated by '$PROPERTY_SEPARATOR'!" + ) + } + string + .split(PROPERTY_SEPARATOR) + .map { word => + val pairs = word.split(KEY_VALUE_SEPARATOR) + pairs(0) -> pairs(1) + } + .toMap + } + +} diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala index b713c7d7..b77d7a9e 100644 --- a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala @@ -4,6 +4,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.{Map => MMap} import com.exasol.cloudetl.common.AbstractProperties +import com.exasol.cloudetl.common.CommonProperties import io.confluent.kafka.serializers.AbstractKafkaAvroSerDeConfig import io.confluent.kafka.serializers.KafkaAvroDeserializer @@ -189,18 +190,8 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) /** * A companion object for [[KafkaConsumerProperties]] class. */ -object KafkaConsumerProperties { - - /** - * A line separator string used for creating key-value property - * strings. - */ - private[kafka] final val PROPERTY_SEPARATOR: String = ";" - - /** - * A default separator string used for concatenate key-value pairs. - */ - private[kafka] final val KEY_VALUE_SEPARATOR: String = " -> " +@SuppressWarnings(Array("org.wartremover.warts.Overloading")) +object KafkaConsumerProperties extends CommonProperties { /** * A required property key name for a Kafka topic name to import data @@ -423,24 +414,10 @@ object KafkaConsumerProperties { new KafkaConsumerProperties(params) /** - * Creates [[KafkaConsumerProperties]] from properly separated string. + * Returns [[KafkaConsumerProperties]] from properly separated string. */ - def fromString(string: String): KafkaConsumerProperties = { - if (!string.contains(PROPERTY_SEPARATOR)) { - throw new IllegalArgumentException( - s"The input string is not separated by '$PROPERTY_SEPARATOR'!" - ) - } - val properties = string - .split(PROPERTY_SEPARATOR) - .map { word => - val pairs = word.split(KEY_VALUE_SEPARATOR) - pairs(0) -> pairs(1) - } - .toMap - - new KafkaConsumerProperties(properties) - } + def apply(string: String): KafkaConsumerProperties = + apply(mapFromString(string)) /** * Creates a [[org.apache.kafka.clients.consumer.KafkaConsumer]] from diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportTable.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportTable.scala index 3e1db24b..ab2da7cc 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportTable.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/ExportTable.scala @@ -17,7 +17,7 @@ import com.typesafe.scalalogging.LazyLogging object ExportTable extends LazyLogging { def run(metadata: ExaMetadata, iterator: ExaIterator): Unit = { - val storageProperties = StorageProperties.fromString(iterator.getString(1)) + val storageProperties = StorageProperties(iterator.getString(1)) val bucket = Bucket(storageProperties) val srcColumnNames = iterator.getString(2).split("\\.") val firstColumnIdx = 3 diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportFiles.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportFiles.scala index 34952e1d..7392caae 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportFiles.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportFiles.scala @@ -15,7 +15,7 @@ import org.apache.hadoop.fs.Path object ImportFiles extends LazyLogging { def run(metadata: ExaMetadata, iterator: ExaIterator): Unit = { - val storageProperties = StorageProperties.fromString(iterator.getString(1)) + val storageProperties = StorageProperties(iterator.getString(1)) val fileFormat = storageProperties.getFileFormat() val bucket = Bucket(storageProperties) diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportMetadata.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportMetadata.scala index 852ab938..41c6f178 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportMetadata.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/ImportMetadata.scala @@ -17,7 +17,7 @@ object ImportMetadata extends LazyLogging { + s"with parallelism: ${parallelism.toString}" ) - val storageProperties = StorageProperties.fromString(iterator.getString(1)) + val storageProperties = StorageProperties(iterator.getString(1)) val bucket = Bucket(storageProperties) val paths = bucket.getPaths().filter(p => !p.getName().startsWith("_")) logger.info(s"Total number of files: ${paths.size} in bucket path: $bucketPath") diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala index 04df3f66..3d05fec2 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala @@ -17,7 +17,7 @@ object KafkaImport extends LazyLogging { @SuppressWarnings(Array("org.wartremover.warts.AsInstanceOf")) def run(metadata: ExaMetadata, iterator: ExaIterator): Unit = { - val kafkaProperties = KafkaConsumerProperties.fromString(iterator.getString(0)) + val kafkaProperties = KafkaConsumerProperties(iterator.getString(0)) val partitionId = iterator.getInteger(1) val partitionOffset = iterator.getLong(2) val partitionNextOffset = partitionOffset + 1L diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala index 3f8840fb..618f416d 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadata.scala @@ -17,7 +17,7 @@ object KafkaMetadata extends LazyLogging { @SuppressWarnings(Array("org.wartremover.warts.MutableDataStructures")) def run(metadata: ExaMetadata, iterator: ExaIterator): Unit = { - val kafkaProperties = KafkaConsumerProperties.fromString(iterator.getString(0)) + val kafkaProperties = KafkaConsumerProperties(iterator.getString(0)) val idOffsetPairs: HashMap[JInt, JLong] = HashMap.empty[JInt, JLong] do { diff --git a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala index 32bb1f8d..fd22a371 100644 --- a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala @@ -3,6 +3,7 @@ package com.exasol.cloudetl.storage import java.net.URI import com.exasol.cloudetl.common.AbstractProperties +import com.exasol.cloudetl.common.CommonProperties /** * A specific implementation of @@ -64,18 +65,8 @@ class StorageProperties(private val properties: Map[String, String]) /** * A companion object for [[StorageProperties]] class. */ -object StorageProperties { - - /** - * A line separator string used for creating key-value property - * strings. - */ - private[storage] final val PROPERTY_SEPARATOR: String = ";" - - /** - * A default separator string used for concatenate key-value pairs. - */ - private[storage] final val KEY_VALUE_SEPARATOR: String = " -> " +@SuppressWarnings(Array("org.wartremover.warts.Overloading")) +object StorageProperties extends CommonProperties { /** A required property key name for a bucket path. */ private[storage] final val BUCKET_PATH: String = "BUCKET_PATH" @@ -86,27 +77,12 @@ object StorageProperties { /** An optional property key name for the parallelism. */ private[storage] final val PARALLELISM: String = "PARALLELISM" + /** Returns [[StorageProperties]] from key-value pairs map. */ def apply(params: Map[String, String]): StorageProperties = new StorageProperties(params) - /** - * Creates [[StorageProperties]] from properly separated string. - */ - def fromString(string: String): StorageProperties = { - if (!string.contains(PROPERTY_SEPARATOR)) { - throw new IllegalArgumentException( - s"The input string is not separated by '$PROPERTY_SEPARATOR'!" - ) - } - val properties = string - .split(PROPERTY_SEPARATOR) - .map { word => - val pairs = word.split(KEY_VALUE_SEPARATOR) - pairs(0) -> pairs(1) - } - .toMap - - new StorageProperties(properties) - } + /** Returns [[StorageProperties]] from properly separated string. */ + def apply(string: String): StorageProperties = + apply(mapFromString(string)) } diff --git a/src/test/scala/com/exasol/cloudetl/common/CommonPropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/common/CommonPropertiesTest.scala new file mode 100644 index 00000000..be4296a3 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/common/CommonPropertiesTest.scala @@ -0,0 +1,27 @@ +package com.exasol.cloudetl.common + +import org.scalatest.FunSuite + +class CommonPropertiesTest extends FunSuite { + + test("mapFromString returns key-value map") { + val testData = Map( + ";" -> Map.empty[String, String], + "a -> 1;b -> 2;c -> 3" -> Map("a" -> "1", "b" -> "2", "c" -> "3") + ) + testData.foreach { + case (given, expected) => + assert(CommonProperties.mapFromString(given) === expected) + } + } + + test("mapFromString throws if input is not properly separated") { + val thrown = intercept[IllegalArgumentException] { + CommonProperties.mapFromString("") + } + assert(thrown.getMessage === s"The input string is not separated by ';'!") + } + + private[this] object CommonProperties extends CommonProperties + +} diff --git a/src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerTest.scala b/src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerTest.scala index 339abb7f..bc0775a3 100644 --- a/src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerTest.scala +++ b/src/test/scala/com/exasol/cloudetl/orc/OrcDeserializerTest.scala @@ -13,7 +13,7 @@ class OrcDeserializerTest extends FunSuite { assert(thrown.getMessage === "Orc list type is not supported.") } - test("apply throws f orc type is a map") { + test("apply throws if orc type is a map") { val orcMap = TypeDescription.createMap(TypeDescription.createString, TypeDescription.createString) val thrown = intercept[IllegalArgumentException] { diff --git a/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala index d9103d47..c455adcc 100644 --- a/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala +++ b/src/test/scala/com/exasol/cloudetl/storage/StoragePropertiesTest.scala @@ -57,6 +57,15 @@ class StoragePropertiesTest extends FunSuite with BeforeAndAfterEach { assert(thrown.getMessage === s"Unsupported file format $fileFormat!") } + test("getParallelism returns user provided value") { + properties = Map(StorageProperties.PARALLELISM -> "2") + assert(BaseProperties(properties).getParallelism("default") === "2") + } + + test("getParallelism returns default value if parallelism is not set") { + assert(BaseProperties(properties).getParallelism("nproc()") === "nproc()") + } + test("mkString returns empty string by default") { val str = BaseProperties(properties).mkString() assert(str.isEmpty === true) @@ -69,18 +78,24 @@ class StoragePropertiesTest extends FunSuite with BeforeAndAfterEach { assert(BaseProperties(properties).mkString() === expected) } - test("fromString throws if input string does not contain separator") { + test("apply(map) returns correct StorageProperties") { + properties = Map("a" -> "b") + val baseProperty = BaseProperties(properties) + assert(StorageProperties(properties) === baseProperty) + } + + test("apply(string) throws if input string does not contain separator") { val thrown = intercept[IllegalArgumentException] { - StorageProperties.fromString("") + StorageProperties("") } assert(thrown.getMessage === s"The input string is not separated by ';'!") } - test("fromString returns correct StorageProperties") { + test("apply(string) returns correct StorageProperties") { properties = Map("k3" -> "v3", "k2" -> "v2") val baseProperty = BaseProperties(properties) val mkStringResult = baseProperty.mkString() - assert(StorageProperties.fromString(mkStringResult) === baseProperty) + assert(StorageProperties(mkStringResult) === baseProperty) } private[this] case class BaseProperties(val params: Map[String, String]) From 219b88408f92c8e121deaa18d96fc04ac7563b85 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Tue, 15 Oct 2019 12:42:37 +0200 Subject: [PATCH 12/18] Remove unused methods from Bucket. --- .../com/exasol/cloudetl/bucket/Bucket.scala | 84 ------------------- 1 file changed, 84 deletions(-) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala index 09e42d37..2d311458 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/Bucket.scala @@ -2,8 +2,6 @@ package com.exasol.cloudetl.bucket import java.net.URI -import scala.collection.SortedMap - import com.exasol.cloudetl.storage.StorageProperties import com.exasol.cloudetl.util.FileSystemUtil @@ -113,86 +111,4 @@ object Bucket extends LazyLogging { def apply(params: Map[String, String]): Bucket = apply(StorageProperties(params)) - /** - * Checks whether the optional parameter is available. If it is not - * available returns the default value. - * - * @param params The parameters key value map - * @param key The optional parameter key - * @param defaultValue The default value to return if key not - * available - * @return The the value for the optional key if it exists; otherwise - * return the default value - */ - def optionalParameter(params: Map[String, String], key: String, defaultValue: String): String = - params.get(key).fold(defaultValue)(identity) - - /** - * Checks whether the optional parameter is available. If it is not - * available returns the default value. - * - * @param params The parameters key value map - * @param key The optional parameter key - * @param defaultValue The default value to return if key not - * available - * @return The the value for the optional key if it exists; otherwise - * return the default value - */ - def optionalIntParameter(params: Map[String, String], key: String, defaultValue: Int): Int = - params.get(key).map(_.toInt).fold(defaultValue)(identity) - - /** - * Converts key value pair strings into a single string with - * separators in between. - * - * In the resulting string, key value pairs will be sorted by the - * keys. - * - * @param params The key value parameters map - * @return A single string with separators - */ - def keyValueMapToString(params: Map[String, String]): String = - (SortedMap.empty[String, String] ++ params) - .map { case (k, v) => s"$k$KEY_VALUE_SEPARATOR$v" } - .mkString(PARAMETER_SEPARATOR) - - /** - * This is opposite of [[Bucket#keyValueMapToString]], given a string - * with separators returns a key value pairs map. - * - * @param params The key value parameters map - * @return A single string with separators - */ - def keyValueStringToMap(keyValueString: String): Map[String, String] = - keyValueString - .split(PARAMETER_SEPARATOR) - .map { word => - val kv = word.split(KEY_VALUE_SEPARATOR) - kv(0) -> kv(1) - } - .toMap - - /** - * Checks if the sequence of keys are available in the key value - * parameter map. - */ - private[bucket] def validate(params: Map[String, String], keys: Seq[String]): Unit = - keys.foreach { key => - requiredParam(params, key) - } - - /** - * Checks if the provided key is available in the key value parameter - * map. If it does not exist, throws an - * [[java.lang.IllegalArgumentException]] exception. - */ - def requiredParam(params: Map[String, String], key: String): String = { - val opt = params.get(key) - opt.fold { - throw new IllegalArgumentException(s"The required parameter $key is not defined!") - }(identity) - } - - private[this] final val PARAMETER_SEPARATOR: String = ";" - private[this] final val KEY_VALUE_SEPARATOR: String = ":=:" } From bc057f0c6974c1b6b9c9e400e908829400389f12 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Tue, 15 Oct 2019 13:09:48 +0200 Subject: [PATCH 13/18] Use getString method instead of getAs[String] method. --- .../cloudetl/bucket/AzureAdlsBucket.scala | 6 ++--- .../cloudetl/bucket/AzureBlobBucket.scala | 4 ++-- .../exasol/cloudetl/bucket/GCSBucket.scala | 4 ++-- .../com/exasol/cloudetl/bucket/S3Bucket.scala | 6 ++--- .../cloudetl/common/AbstractProperties.scala | 11 +++++++++ .../kafka/KafkaConsumerProperties.scala | 23 ++++++++----------- .../cloudetl/storage/StorageProperties.scala | 11 +++------ .../common/AbstractPropertiesTest.scala | 15 ++++++++++++ .../cloudetl/source/ParquetSourceTest.scala | 2 +- 9 files changed, 49 insertions(+), 33 deletions(-) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala index 5b1be6ee..7dc8ccb9 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/AzureAdlsBucket.scala @@ -34,9 +34,9 @@ final case class AzureAdlsBucket(path: String, params: StorageProperties) extend validate() val conf = new Configuration() - val clientId = properties.getAs[String](AZURE_CLIENT_ID) - val clientSecret = properties.getAs[String](AZURE_CLIENT_SECRET) - val directoryId = properties.getAs[String](AZURE_DIRECTORY_ID) + val clientId = properties.getString(AZURE_CLIENT_ID) + val clientSecret = properties.getString(AZURE_CLIENT_SECRET) + val directoryId = properties.getString(AZURE_DIRECTORY_ID) val tokenEndpoint = s"https://login.microsoftonline.com/$directoryId/oauth2/token" conf.set("fs.adl.impl", classOf[org.apache.hadoop.fs.adl.AdlFileSystem].getName) conf.set("fs.AbstractFileSystem.adl.impl", classOf[org.apache.hadoop.fs.adl.Adl].getName) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala index a60fa872..51bf659c 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/AzureBlobBucket.scala @@ -32,8 +32,8 @@ final case class AzureBlobBucket(path: String, params: StorageProperties) extend validate() val conf = new Configuration() - val accountName = properties.getAs[String](AZURE_ACCOUNT_NAME) - val accountSecretKey = properties.getAs[String](AZURE_SECRET_KEY) + val accountName = properties.getString(AZURE_ACCOUNT_NAME) + val accountSecretKey = properties.getString(AZURE_SECRET_KEY) conf.set("fs.azure", classOf[org.apache.hadoop.fs.azure.NativeAzureFileSystem].getName) conf.set("fs.wasb.impl", classOf[org.apache.hadoop.fs.azure.NativeAzureFileSystem].getName) conf.set("fs.wasbs.impl", classOf[org.apache.hadoop.fs.azure.NativeAzureFileSystem].getName) diff --git a/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala index f51dbc92..d85d107e 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/GCSBucket.scala @@ -35,10 +35,10 @@ final case class GCSBucket(path: String, params: StorageProperties) extends Buck val conf = new Configuration() conf.set("fs.gs.impl", classOf[com.google.cloud.hadoop.fs.gcs.GoogleHadoopFileSystem].getName) conf.setBoolean("fs.gs.auth.service.account.enable", true) - conf.set("fs.gs.project.id", properties.getAs[String](GCS_PROJECT_ID)) + conf.set("fs.gs.project.id", properties.getString(GCS_PROJECT_ID)) conf.set( "fs.gs.auth.service.account.json.keyfile", - properties.getAs[String](GCS_KEYFILE_PATH) + properties.getString(GCS_KEYFILE_PATH) ) conf diff --git a/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala b/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala index 2abe8321..0e8c8185 100644 --- a/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala +++ b/src/main/scala/com/exasol/cloudetl/bucket/S3Bucket.scala @@ -33,9 +33,9 @@ final case class S3Bucket(path: String, params: StorageProperties) extends Bucke val conf = new Configuration() conf.set("fs.file.impl", classOf[org.apache.hadoop.fs.LocalFileSystem].getName) conf.set("fs.s3a.impl", classOf[org.apache.hadoop.fs.s3a.S3AFileSystem].getName) - conf.set("fs.s3a.endpoint", properties.getAs[String](S3_ENDPOINT)) - conf.set("fs.s3a.access.key", properties.getAs[String](S3_ACCESS_KEY)) - conf.set("fs.s3a.secret.key", properties.getAs[String](S3_SECRET_KEY)) + conf.set("fs.s3a.endpoint", properties.getString(S3_ENDPOINT)) + conf.set("fs.s3a.access.key", properties.getString(S3_ACCESS_KEY)) + conf.set("fs.s3a.secret.key", properties.getString(S3_SECRET_KEY)) conf } diff --git a/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala b/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala index 7fff3645..1a9ff777 100644 --- a/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/common/AbstractProperties.scala @@ -48,6 +48,17 @@ abstract class AbstractProperties(private val properties: Map[String, String]) { final def get(key: String): Option[String] = properties.get(key) + /** + * Returns the value of the key as a String. + * + * @throws java.lang.IllegalArgumentException If key does not exist. + */ + @throws[IllegalArgumentException]("If key does not exist.") + final def getString(key: String): String = + get(key).fold { + throw new IllegalArgumentException(s"Please provide a value for the $key property!") + }(identity) + /** * Returns the count of the key-value properties. */ diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala index b77d7a9e..8b8409ee 100644 --- a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala @@ -30,18 +30,18 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) import KafkaConsumerProperties._ final def getBootstrapServers(): String = - getAs[String](BOOTSTRAP_SERVERS.userPropertyName) + getString(BOOTSTRAP_SERVERS.userPropertyName) final def getGroupId(): String = get(GROUP_ID.userPropertyName).fold(GROUP_ID.defaultValue)(identity) /** Returns the user provided topic name. */ final def getTopics(): String = - getAs[String](TOPICS) + getString(TOPICS) /** Returns the user provided Exasol table name. */ final def getTableName(): String = - getAs[String](TABLE_NAME) + getString(TABLE_NAME) final def getPollTimeoutMs(): Long = get(POLL_TIMEOUT_MS).fold(POLL_TIMEOUT_MS_DEFAULT_VALUE)(_.asInstanceOf[Long]) @@ -62,7 +62,7 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) /** Returns the user provided schema registry url property. */ final def getSchemaRegistryUrl(): String = - getAs[String](SCHEMA_REGISTRY_URL.userPropertyName) + getString(SCHEMA_REGISTRY_URL.userPropertyName) /** * Returns {@code MAX_POLL_RECORDS} property value if provided, @@ -91,35 +91,35 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) * Returns the user provided {@code SSL_KEY_PASSWORD} property value. */ final def getSSLKeyPassword(): String = - getAs[String](SSL_KEY_PASSWORD.userPropertyName) + getString(SSL_KEY_PASSWORD.userPropertyName) /** * Returns the user provided {@code SSL_KEYSTORE_PASSWORD} property * value. */ final def getSSLKeystorePassword(): String = - getAs[String](SSL_KEYSTORE_PASSWORD.userPropertyName) + getString(SSL_KEYSTORE_PASSWORD.userPropertyName) /** * Returns the user provided {@code SSL_KEYSTORE_LOCATION} property * value. */ final def getSSLKeystoreLocation(): String = - getAs[String](SSL_KEYSTORE_LOCATION.userPropertyName) + getString(SSL_KEYSTORE_LOCATION.userPropertyName) /** * Returns the user provided {@code SSL_TRUSTSTORE_PASSWORD} property * value. */ final def getSSLTruststorePassword(): String = - getAs[String](SSL_TRUSTSTORE_PASSWORD.userPropertyName) + getString(SSL_TRUSTSTORE_PASSWORD.userPropertyName) /** * Returns the user provided {@code SSL_TRUSTSTORE_LOCATION} property * value. */ final def getSSLTruststoreLocation(): String = - getAs[String](SSL_TRUSTSTORE_LOCATION.userPropertyName) + getString(SSL_TRUSTSTORE_LOCATION.userPropertyName) /** * Returns {@code SSL_ENDPOINT_IDENTIFICATION_ALGORITHM} property @@ -171,11 +171,6 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) props.toMap.asInstanceOf[Map[String, AnyRef]].asJava } - final def getAs[T](key: String): T = - get(key).fold { - throw new IllegalArgumentException(s"Please provide a value for the $key property!") - }(_.asInstanceOf[T]) - /** * Returns a string value of key-value property pairs. * diff --git a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala index fd22a371..a7a79c20 100644 --- a/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/storage/StorageProperties.scala @@ -23,7 +23,7 @@ class StorageProperties(private val properties: Map[String, String]) * stored. */ final def getStoragePath(): String = - getAs[String](BUCKET_PATH) + getString(BUCKET_PATH) /** * Returns the storage path scheme. @@ -36,7 +36,7 @@ class StorageProperties(private val properties: Map[String, String]) /** Returns the [[FileFormat]] file format. */ final def getFileFormat(): FileFormat = - FileFormat(getAs[String](DATA_FORMAT)) + FileFormat(getString(DATA_FORMAT)) /** * Returns the number of partitions provided as user property. @@ -46,15 +46,10 @@ class StorageProperties(private val properties: Map[String, String]) final def getParallelism(defaultValue: => String): String = get(PARALLELISM).fold(defaultValue)(identity) - final def getAs[T](key: String): T = - get(key).fold { - throw new IllegalArgumentException(s"Please provide a value for the $key property!") - }(_.asInstanceOf[T]) - /** * Returns a string value of key-value property pairs. * - * The resulting string is sorted by keys ordering. + * The returned string is sorted by keys ordering. */ @SuppressWarnings(Array("org.wartremover.warts.Overloading")) final def mkString(): String = diff --git a/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala index 105b97bd..0d1826e2 100644 --- a/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala +++ b/src/test/scala/com/exasol/cloudetl/common/AbstractPropertiesTest.scala @@ -67,6 +67,21 @@ class AbstractPropertiesTest extends FunSuite with BeforeAndAfterEach { assert(BaseProperties(properties).get("key") === Option("value")) } + test("getString returns value as String") { + properties = Map("key" -> "value") + assert(BaseProperties(properties).getString("key") === "value") + } + + test("getString throws if key does not exits") { + val key = "key" + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).getString(key) + } + assert( + thrown.getMessage === s"Please provide a value for the $key property!" + ) + } + test("size returns zero by default") { assert(BaseProperties(properties).size() === 0) } diff --git a/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala index db3be937..0e5f7167 100644 --- a/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala +++ b/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala @@ -37,7 +37,7 @@ class ParquetSourceTest extends SourceTest { assert(getRecordsCount(filePath) === 500) } - test("stream reaturns count of records from PARQUET files") { + test("stream returns count of records from PARQUET files") { val filePattern = Paths.get(s"$resourceDir/sales_positions*.parquet") assert(getRecordsCount(filePattern) === 1005) } From 513e76dfd772f5d3bfaa07b0970315e0f75cb2c8 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Tue, 15 Oct 2019 13:44:06 +0200 Subject: [PATCH 14/18] Parameterize Config class in Kafka consumer properties. --- .../kafka/KafkaConsumerProperties.scala | 145 +++++++++++++----- 1 file changed, 104 insertions(+), 41 deletions(-) diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala index 8b8409ee..c353b6bf 100644 --- a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala @@ -29,9 +29,14 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) import KafkaConsumerProperties._ + /** Returns user provided Kafka bootstrap servers string. */ final def getBootstrapServers(): String = getString(BOOTSTRAP_SERVERS.userPropertyName) + /** + * Returns user provided group id, if it is not provided by user + * returns default value. + */ final def getGroupId(): String = get(GROUP_ID.userPropertyName).fold(GROUP_ID.defaultValue)(identity) @@ -39,18 +44,42 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) final def getTopics(): String = getString(TOPICS) - /** Returns the user provided Exasol table name. */ + /** + * Returns the user provided Exasol table name; otherwise returns + * default value. + */ final def getTableName(): String = getString(TABLE_NAME) + /** + * Returns poll timeout millisecords if provided by user; otherwise + * returns default value. + * + * @throws java.lang.NumberFormatException If value is not a Long. + */ + @throws[NumberFormatException]("If value is not a Long.") final def getPollTimeoutMs(): Long = - get(POLL_TIMEOUT_MS).fold(POLL_TIMEOUT_MS_DEFAULT_VALUE)(_.asInstanceOf[Long]) - - final def getMaxRecordsPerRun(): Int = - get(MAX_RECORDS_PER_RUN).fold(MAX_RECORDS_PER_RUN_DEFAULT_VALUE)(_.asInstanceOf[Int]) + get(POLL_TIMEOUT_MS.userPropertyName).fold(POLL_TIMEOUT_MS.defaultValue)(_.toLong) + /** + * Returns minimum records per run property value when provided by + * user; otherwise returns default value. + * + * @throws java.lang.NumberFormatException If value is not an Int. + */ + @throws[NumberFormatException]("If value is not an Int.") final def getMinRecordsPerRun(): Int = - get(MIN_RECORDS_PER_RUN).fold(MIN_RECORDS_PER_RUN_DEFAULT_VALUE)(_.asInstanceOf[Int]) + get(MIN_RECORDS_PER_RUN.userPropertyName).fold(MIN_RECORDS_PER_RUN.defaultValue)(_.toInt) + + /** + * Returns maximum records per run property value when provided by + * user; otherwise returns default value. + * + * @throws java.lang.NumberFormatException If value is not an Int. + */ + @throws[NumberFormatException]("If value is not an Int.") + final def getMaxRecordsPerRun(): Int = + get(MAX_RECORDS_PER_RUN.userPropertyName).fold(MAX_RECORDS_PER_RUN.defaultValue)(_.toInt) /** Checks if the {@code SSL_ENABLED} property is set. */ final def isSSLEnabled(): Boolean = @@ -188,6 +217,20 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) @SuppressWarnings(Array("org.wartremover.warts.Overloading")) object KafkaConsumerProperties extends CommonProperties { + /** + * Internal configuration helper class. + * + * @param userPropertyName A UDF user provided property key name + * @param kafkaPropertyName An equivalent property in Kafka + * configuration that maps user property key name + * @param defaultValue A default value for the property key name + */ + private[kafka] final case class Config[T]( + val userPropertyName: String, + val kafkaPropertyName: String, + val defaultValue: T + ) + /** * A required property key name for a Kafka topic name to import data * from. @@ -200,15 +243,6 @@ object KafkaConsumerProperties extends CommonProperties { */ private[kafka] final val TABLE_NAME: String = "TABLE_NAME" - private[kafka] final val POLL_TIMEOUT_MS: String = "POLL_TIMEOUT_MS" - private[kafka] final val POLL_TIMEOUT_MS_DEFAULT_VALUE: Long = 30000L - - private[kafka] final val MAX_RECORDS_PER_RUN: String = "MAX_RECORDS_PER_RUN" - private[kafka] final val MAX_RECORDS_PER_RUN_DEFAULT_VALUE: Int = 1000000 - - private[kafka] final val MIN_RECORDS_PER_RUN: String = "MIN_RECORDS_PER_RUN" - private[kafka] final val MIN_RECORDS_PER_RUN_DEFAULT_VALUE: Int = 100 - /** * An optional property key name to set SSL secure connections to * Kafka cluster. @@ -216,25 +250,54 @@ object KafkaConsumerProperties extends CommonProperties { private[kafka] val SSL_ENABLED: String = "SSL_ENABLED" /** - * Below are relavant Kafka consumer configuration parameters are - * defined. + * A number of milliseconds to wait for Kafka consumer {@code poll} to + * return any data. + */ + private[kafka] final val POLL_TIMEOUT_MS = Config[Long]( + "POLL_TIMEOUT_MS", + "", + 30000L // scalastyle:ignore magic.number + ) + + /** + * An upper bound on the minimum number of records to consume per UDF + * run. * - * See [[https://kafka.apache.org/documentation.html#consumerconfigs]] + * That is, if the {@code poll} returns fewer records than this + * number, consume them and finish the process. Otherwise, continue + * polling more data until the total number of records reaches + * [[MAX_RECORDS_PER_RUN]]. + * + * See [[MAX_RECORDS_PER_RUN]]. */ + private[kafka] final val MIN_RECORDS_PER_RUN = Config[Int]( + "MIN_RECORDS_PER_RUN", + "", + 100 // scalastyle:ignore magic.number + ) + /** - * Internal configuration helper class. + * An lower bound on the maximum number of records to consumer per UDF + * run. * - * @param userPropertyName A UDF user provided property key name - * @param kafkaPropertyName An equivalent property in Kafka - * configuration that maps user property key name - * @param defaultValue A default value for the property key name + * When the returned number of records from {@code poll} is more than + * [[MIN_RECORDS_PER_RUN]], it continues polling for more records + * until total number reaches this number. + * + * See [[MIN_RECORDS_PER_RUN]]. */ - private[kafka] final case class Config( - val userPropertyName: String, - val kafkaPropertyName: String, - val defaultValue: String + private[kafka] final val MAX_RECORDS_PER_RUN = Config[Int]( + "MAX_RECORDS_PER_RUN", + "", + 1000000 // scalastyle:ignore magic.number ) + /** + * Below are relavant Kafka consumer configuration parameters are + * defined. + * + * See [[https://kafka.apache.org/documentation.html#consumerconfigs]] + */ /** * This is the {@code enable.auto.commit} configuration setting. * @@ -243,21 +306,21 @@ object KafkaConsumerProperties extends CommonProperties { * by default, since we manage the offset commits ourselves in the * Exasol table. */ - private[kafka] val ENABLE_AUTO_COMMIT: Config = Config( + private[kafka] val ENABLE_AUTO_COMMIT = Config[String]( "ENABLE_AUTO_COMMIT", ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false" ) /** - * This is the `bootstrap.servers` configuration setting. + * This is the {@code bootstrap.servers} configuration setting. * * A list of host and port pairs to use for establishing the initial * connection to the Kafka cluster. * * It is a required property that should be provided by the user. */ - private[kafka] val BOOTSTRAP_SERVERS: Config = Config( + private[kafka] val BOOTSTRAP_SERVERS = Config[String]( "BOOTSTRAP_SERVERS", ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "" @@ -269,7 +332,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is a unique string that identifies the consumer group this * consumer belongs to. */ - private[kafka] val GROUP_ID: Config = Config( + private[kafka] val GROUP_ID = Config[String]( "GROUP_ID", ConsumerConfig.GROUP_ID_CONFIG, "EXASOL_KAFKA_UDFS_CONSUMERS" @@ -281,7 +344,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is the maximum number of records returned in a single call to * poll() function. Default value is `500`. */ - private[kafka] val MAX_POLL_RECORDS: Config = Config( + private[kafka] val MAX_POLL_RECORDS = Config[String]( "MAX_POLL_RECORDS", ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "500" @@ -293,7 +356,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is the minimum amount of data the server should return for a * fetch request. Default value is `1`. */ - private[kafka] val FETCH_MIN_BYTES: Config = Config( + private[kafka] val FETCH_MIN_BYTES = Config[String]( "FETCH_MIN_BYTES", ConsumerConfig.FETCH_MIN_BYTES_CONFIG, "1" @@ -305,7 +368,7 @@ object KafkaConsumerProperties extends CommonProperties { * The Avro value deserializer will be used when user sets this * property value. */ - private[kafka] val SCHEMA_REGISTRY_URL: Config = Config( + private[kafka] val SCHEMA_REGISTRY_URL = Config[String]( "SCHEMA_REGISTRY_URL", AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, "" @@ -318,7 +381,7 @@ object KafkaConsumerProperties extends CommonProperties { * [[SSL_ENABLED]] is set to {@code true}. Default value is * [[SslConfigs.DEFAULT_SSL_PROTOCOL]]. */ - private[kafka] val SECURITY_PROTOCOL: Config = Config( + private[kafka] val SECURITY_PROTOCOL = Config[String]( "SECURITY_PROTOCOL", CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SslConfigs.DEFAULT_SSL_PROTOCOL @@ -331,7 +394,7 @@ object KafkaConsumerProperties extends CommonProperties { * file. It is required property when [[SSL_ENABLED]] is set to {@code * true}. */ - private[kafka] val SSL_KEY_PASSWORD: Config = Config( + private[kafka] val SSL_KEY_PASSWORD = Config[String]( "SSL_KEY_PASSWORD", SslConfigs.SSL_KEY_PASSWORD_CONFIG, "" @@ -343,7 +406,7 @@ object KafkaConsumerProperties extends CommonProperties { * It the store password for the keystore file. It is required * property when [[SSL_ENABLED]] is set to {@code true}. */ - private[kafka] val SSL_KEYSTORE_PASSWORD: Config = Config( + private[kafka] val SSL_KEYSTORE_PASSWORD = Config[String]( "SSL_KEYSTORE_PASSWORD", SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "" @@ -356,7 +419,7 @@ object KafkaConsumerProperties extends CommonProperties { * property when [[SSL_ENABLED]] is set to {@code true} and can be * used for two-way authentication for the clients. */ - private[kafka] val SSL_KEYSTORE_LOCATION: Config = Config( + private[kafka] val SSL_KEYSTORE_LOCATION = Config[String]( "SSL_KEYSTORE_LOCATION", SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, "" @@ -368,7 +431,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is the password for the truststore file, and required property * when [[SSL_ENABLED]] is set to {@code true}. */ - private[kafka] val SSL_TRUSTSTORE_PASSWORD: Config = Config( + private[kafka] val SSL_TRUSTSTORE_PASSWORD = Config[String]( "SSL_TRUSTSTORE_PASSWORD", SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "" @@ -380,7 +443,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is the location of the truststore file, and required property * when [[SSL_ENABLED]] is set to {@code true}. */ - private[kafka] val SSL_TRUSTSTORE_LOCATION: Config = Config( + private[kafka] val SSL_TRUSTSTORE_LOCATION = Config[String]( "SSL_TRUSTSTORE_LOCATION", SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "" @@ -395,7 +458,7 @@ object KafkaConsumerProperties extends CommonProperties { * is set to {@code true}. Default value is * [[SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM]]. */ - private[kafka] val SSL_ENDPOINT_IDENTIFICATION_ALGORITHM: Config = Config( + private[kafka] val SSL_ENDPOINT_IDENTIFICATION_ALGORITHM = Config[String]( "SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM From de4b42e06eb1536cde6f8c26031668e71d22a4b0 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Mon, 21 Oct 2019 11:24:20 +0200 Subject: [PATCH 15/18] Adds Kafka integration tests using SchemaRegistryEmbeddedKafka library. The [embedded-kafka-schema-registry](https://github.com/embeddedkafka/embedded-kafka-schema-registry) library provides in-memory instances of Kafka, Zookeeper and Confluent Schema Registry that makes it easy to run tests against. Furthermore, refactors and adds more unit tests. --- project/Dependencies.scala | 14 +- project/Settings.scala | 3 +- .../cloudetl/KafkaIntegrationTest.scala | 98 +++++++++ .../kafka/KafkaConsumerPropertiesIT.scala | 14 ++ .../scriptclasses/KafkaImportIT.scala | 84 ++++++++ .../scriptclasses/KafkaMetadataIT.scala | 74 +++++++ .../scala/com/exasol/cloudetl/data/Row.scala | 24 ++- .../kafka/KafkaConsumerProperties.scala | 34 +-- .../cloudetl/source/ParquetSource.scala | 9 +- .../com/exasol/cloudetl/data/RowTest.scala | 201 ++++++++++++++++++ .../kafka/KafkaConsumerPropertiesTest.scala | 39 +++- .../parquet/ParquetWriteOptionsTest.scala | 94 ++++++++ .../scriptclasses/KafkaPathTest.scala | 82 +++++++ .../cloudetl/scriptclasses/PathTest.scala | 5 +- .../cloudetl/source/AbstractSourceTest.scala | 50 +++++ .../cloudetl/source/AvroSourceTest.scala | 10 +- .../cloudetl/source/OrcSourceTest.scala | 10 +- .../cloudetl/source/ParquetSourceTest.scala | 10 +- .../exasol/cloudetl/source/SourceTest.scala | 44 +--- 19 files changed, 820 insertions(+), 79 deletions(-) create mode 100644 src/it/scala/com/exasol/cloudetl/KafkaIntegrationTest.scala create mode 100644 src/it/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesIT.scala create mode 100644 src/it/scala/com/exasol/cloudetl/scriptclasses/KafkaImportIT.scala create mode 100644 src/it/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadataIT.scala create mode 100644 src/test/scala/com/exasol/cloudetl/data/RowTest.scala create mode 100644 src/test/scala/com/exasol/cloudetl/parquet/ParquetWriteOptionsTest.scala create mode 100644 src/test/scala/com/exasol/cloudetl/scriptclasses/KafkaPathTest.scala create mode 100644 src/test/scala/com/exasol/cloudetl/source/AbstractSourceTest.scala diff --git a/project/Dependencies.scala b/project/Dependencies.scala index fe99e2c8..cea9cd90 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -1,6 +1,7 @@ package com.exasol.cloudetl.sbt import sbt._ +import sbt.librarymanagement.InclExclRule /** A list of required dependencies */ object Dependencies { @@ -42,9 +43,20 @@ object Dependencies { /** Test dependencies only required in `test` */ private val TestDependencies: Seq[ModuleID] = Seq( "org.scalatest" %% "scalatest" % "3.0.5", - "org.mockito" % "mockito-core" % "2.23.4" + "org.mockito" % "mockito-core" % "2.23.4", + "io.github.embeddedkafka" %% "embedded-kafka-schema-registry" % "5.3.0" ).map(_ % Test) + lazy val ExcludedDependencies: Seq[InclExclRule] = Seq( + ExclusionRule("org.ow2.asm", "asm"), + ExclusionRule("javax.ws.rs", "jsr311-api"), + ExclusionRule("com.sun.jersey", "jersey-core"), + ExclusionRule("com.sun.jersey", "jersey-server"), + ExclusionRule("com.sun.jersey", "jersey-json"), + ExclusionRule("javax.servlet", "servlet-api"), + ExclusionRule("javax.servlet.jsp", "jsp-api") + ) + /** The list of all dependencies for the connector */ lazy val AllDependencies: Seq[ModuleID] = CoreDependencies ++ TestDependencies diff --git a/project/Settings.scala b/project/Settings.scala index 2f81dc14..83d5d5bb 100644 --- a/project/Settings.scala +++ b/project/Settings.scala @@ -30,7 +30,8 @@ object Settings { compileOrder in Compile := CompileOrder.JavaThenScala, // Dependency settings resolvers ++= Dependencies.Resolvers, - libraryDependencies ++= Dependencies.AllDependencies + libraryDependencies ++= Dependencies.AllDependencies, + excludeDependencies ++= Dependencies.ExcludedDependencies ) def miscSettings(): Seq[Setting[_]] = Seq( diff --git a/src/it/scala/com/exasol/cloudetl/KafkaIntegrationTest.scala b/src/it/scala/com/exasol/cloudetl/KafkaIntegrationTest.scala new file mode 100644 index 00000000..2ad75a84 --- /dev/null +++ b/src/it/scala/com/exasol/cloudetl/KafkaIntegrationTest.scala @@ -0,0 +1,98 @@ +package com.exasol.cloudetl.kafka + +import com.exasol.ExaIterator + +import net.manub.embeddedkafka.schemaregistry.EmbeddedKafka +import org.apache.avro.AvroRuntimeException +import org.apache.avro.Schema +import org.apache.avro.specific.SpecificRecordBase +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfterEach +import org.scalatest.FunSuite +import org.scalatest.mockito.MockitoSugar + +trait KafkaIntegrationTest + extends FunSuite + with BeforeAndAfterEach + with MockitoSugar + with EmbeddedKafka { + + val topic = "exasol-kafka-topic" + + val properties = Map( + "BOOTSTRAP_SERVERS" -> "localhost:6001", + "SCHEMA_REGISTRY_URL" -> "http://localhost:6002", + "TOPICS" -> topic, + "TABLE_NAME" -> "exasolTable" + ) + + override final def beforeEach(): Unit = { + EmbeddedKafka.start() + () + } + + override final def afterEach(): Unit = { + EmbeddedKafka.stop() + () + } + + final def mockExasolIterator( + params: Map[String, String], + partitions: Seq[Int], + offsets: Seq[Long] + ): ExaIterator = { + val mockedIterator = mock[ExaIterator] + when(mockedIterator.getString(0)).thenReturn(KafkaConsumerProperties(params).mkString()) + + val bHead :: bTail = Seq.fill(partitions.size - 1)(true) ++ Seq(false) + when(mockedIterator.next()).thenReturn(bHead, bTail: _*) + + val pHead :: pTail = partitions.map(new java.lang.Integer(_)) + when(mockedIterator.getInteger(1)).thenReturn(pHead, pTail: _*) + + val oHead :: oTail = offsets.map(new java.lang.Long(_)) + when(mockedIterator.getLong(2)).thenReturn(oHead, oTail: _*) + + mockedIterator + } + + private[this] val avroRecordSchema = + new Schema.Parser().parse(s"""{ + | "namespace": "com.exasol.cloudetl", + | "type": "record", + | "name": "AvroRecordSchemaForIT", + | "fields": [ + | {"name": "col_str", "type": "string"}, + | {"name": "col_int", "type": "int"}, + | {"name": "col_long", "type": "long"} + | ] + |}""".stripMargin) + + case class AvroRecord(var col_str: String, var col_int: Int, var col_long: Long) + extends SpecificRecordBase { + def this() = this("", 0, 0) + + override def get(index: Int): AnyRef = index match { + case 0 => col_str + case 1 => col_int.asInstanceOf[AnyRef] + case 2 => col_long.asInstanceOf[AnyRef] + case _ => throw new AvroRuntimeException(s"Unknown index $index!") + } + + override def put(index: Int, value: Any): Unit = index match { + case 0 => + col_str = value match { + case (utf8: org.apache.avro.util.Utf8) => utf8.toString + case _ => value.asInstanceOf[String] + } + case 1 => + col_int = value.asInstanceOf[Int] + case 2 => + col_long = value.asInstanceOf[Long] + case _ => throw new AvroRuntimeException(s"Unknown index $index!") + } + + override def getSchema(): Schema = avroRecordSchema + } + +} diff --git a/src/it/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesIT.scala b/src/it/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesIT.scala new file mode 100644 index 00000000..15d899eb --- /dev/null +++ b/src/it/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesIT.scala @@ -0,0 +1,14 @@ +package com.exasol.cloudetl.kafka + +import org.apache.avro.generic.GenericRecord +import org.apache.kafka.clients.consumer.KafkaConsumer + +@SuppressWarnings(Array("org.wartremover.warts.IsInstanceOf")) +class KafkaConsumerPropertiesIT extends KafkaIntegrationTest { + + test("build returns a KafkaConsumer[String, GenericRecord]") { + val kafkaConsumer = KafkaConsumerProperties(properties).build() + assert(kafkaConsumer.isInstanceOf[KafkaConsumer[String, GenericRecord]]) + } + +} diff --git a/src/it/scala/com/exasol/cloudetl/scriptclasses/KafkaImportIT.scala b/src/it/scala/com/exasol/cloudetl/scriptclasses/KafkaImportIT.scala new file mode 100644 index 00000000..3924fa39 --- /dev/null +++ b/src/it/scala/com/exasol/cloudetl/scriptclasses/KafkaImportIT.scala @@ -0,0 +1,84 @@ +package com.exasol.cloudetl.scriptclasses + +import java.lang.{Integer => JInt} +import java.lang.{Long => JLong} + +import com.exasol.ExaMetadata +import com.exasol.cloudetl.kafka.KafkaIntegrationTest + +import org.mockito.ArgumentMatchers._ +import org.mockito.Mockito.times +import org.mockito.Mockito.verify + +class KafkaImportIT extends KafkaIntegrationTest { + + test("run emits records from starting initial offset") { + createCustomTopic(topic) + publishToKafka(topic, AvroRecord("abc", 3, 13)) + publishToKafka(topic, AvroRecord("hello", 4, 14)) + + val iter = mockExasolIterator(properties, Seq(0), Seq(-1)) + KafkaImport.run(mock[ExaMetadata], iter) + + verify(iter, times(2)).emit(Seq(any[Object]): _*) + verify(iter, times(2)).emit( + anyInt().asInstanceOf[JInt], + anyLong().asInstanceOf[JLong], + anyString(), + anyInt().asInstanceOf[JInt], + anyLong().asInstanceOf[JLong] + ) + verify(iter, times(1)).emit(new JInt(0), new JLong(0), "abc", new JInt(3), new JLong(13)) + verify(iter, times(1)).emit(new JInt(0), new JLong(1), "hello", new JInt(4), new JLong(14)) + } + + test("run emits records starting from provided offset") { + createCustomTopic(topic) + publishToKafka(topic, AvroRecord("abc", 3, 13)) + publishToKafka(topic, AvroRecord("hello", 4, 14)) + publishToKafka(topic, AvroRecord("def", 7, 17)) + publishToKafka(topic, AvroRecord("xyz", 13, 23)) + + // records at 0, 1 are already read, committed + val iter = mockExasolIterator(properties, Seq(0), Seq(1)) + KafkaImport.run(mock[ExaMetadata], iter) + + verify(iter, times(2)).emit(Seq(any[Object]): _*) + verify(iter, times(2)).emit( + anyInt().asInstanceOf[JInt], + anyLong().asInstanceOf[JLong], + anyString(), + anyInt().asInstanceOf[JInt], + anyLong().asInstanceOf[JLong] + ) + verify(iter, times(1)).emit(new JInt(0), new JLong(2), "def", new JInt(7), new JLong(17)) + verify(iter, times(1)).emit(new JInt(0), new JLong(3), "xyz", new JInt(13), new JLong(23)) + } + + test("run emits records within min / max records per run") { + val newProperties = properties ++ Map( + "MAX_POLL_RECORDS" -> "2", + "MIN_RECORDS_PER_RUN" -> "2", + "MAX_RECORDS_PER_RUN" -> "4" + ) + createCustomTopic(topic) + publishToKafka(topic, AvroRecord("abc", 3, 13)) + publishToKafka(topic, AvroRecord("hello", 4, 14)) + publishToKafka(topic, AvroRecord("def", 7, 17)) + publishToKafka(topic, AvroRecord("xyz", 13, 23)) + + // comsumer in two batches each with 2 records + val iter = mockExasolIterator(newProperties, Seq(0), Seq(-1)) + KafkaImport.run(mock[ExaMetadata], iter) + + verify(iter, times(4)).emit(Seq(any[Object]): _*) + verify(iter, times(4)).emit( + anyInt().asInstanceOf[JInt], + anyLong().asInstanceOf[JLong], + anyString(), + anyInt().asInstanceOf[JInt], + anyLong().asInstanceOf[JLong] + ) + } + +} diff --git a/src/it/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadataIT.scala b/src/it/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadataIT.scala new file mode 100644 index 00000000..e46861d1 --- /dev/null +++ b/src/it/scala/com/exasol/cloudetl/scriptclasses/KafkaMetadataIT.scala @@ -0,0 +1,74 @@ +package com.exasol.cloudetl.scriptclasses + +import java.lang.{Integer => JInt} +import java.lang.{Long => JLong} + +import com.exasol.ExaMetadata +import com.exasol.cloudetl.kafka.KafkaIntegrationTest + +import org.mockito.ArgumentMatchers.anyInt +import org.mockito.ArgumentMatchers.anyLong +import org.mockito.Mockito._ + +@SuppressWarnings( + Array("org.wartremover.warts.AsInstanceOf", "org.wartremover.contrib.warts.SymbolicName") +) +class KafkaMetadataIT extends KafkaIntegrationTest { + + // Default case where Exasol table is empty. + test("run emits default partitionId maxOffset pairs with single topic partition") { + val iter = mockExasolIterator(properties, Seq(0), Seq(-1)) + KafkaMetadata.run(mock[ExaMetadata], iter) + verify(iter, times(1)).emit(anyInt().asInstanceOf[JInt], anyLong().asInstanceOf[JLong]) + verify(iter, times(1)).emit(new JInt(0), new JLong(-1)) + } + + // Default case where Exasol table is empty. + test("run emits default partitionId maxOffset pairs with more topic partitions") { + createCustomTopic(topic, partitions = 3) + val iter = mockExasolIterator(properties, Seq(0), Seq(-1)) + KafkaMetadata.run(mock[ExaMetadata], iter) + verify(iter, times(3)).emit(anyInt().asInstanceOf[JInt], anyLong().asInstanceOf[JLong]) + Seq(0, 1, 2).foreach { partitionId => + verify(iter, times(1)).emit(new JInt(partitionId), new JLong(-1)) + } + } + + test("run emits partitionId maxOffset pairs with additional topic partitions") { + createCustomTopic(topic, partitions = 3) + val partitions = Seq(0, 1) + val offsets = Seq(3L, 4L) + val iter = mockExasolIterator(properties, partitions, offsets) + KafkaMetadata.run(mock[ExaMetadata], iter) + + verify(iter, times(3)).emit(anyInt().asInstanceOf[JInt], anyLong().asInstanceOf[JLong]) + partitions.zip(offsets).foreach { + case (partitionId, maxOffset) => + verify(iter, times(1)).emit(new JInt(partitionId), new JLong(maxOffset)) + } + verify(iter, times(1)).emit(new JInt(2), new JLong(-1)) + } + + // Do not emit partitionId maxOffset pairs if partitionId is not + // available in topic partitions + test("run emits partitionId maxOffset pairs with fewer topic partitions") { + createCustomTopic(topic, partitions = 2) + val iter = mockExasolIterator(properties, Seq(1, 3), Seq(7, 17)) + KafkaMetadata.run(mock[ExaMetadata], iter) + + verify(iter, times(2)).emit(anyInt().asInstanceOf[JInt], anyLong().asInstanceOf[JLong]) + verify(iter, times(1)).emit(new JInt(0), new JLong(-1)) + verify(iter, times(1)).emit(new JInt(1), new JLong(7)) + } + + test("run throws if it cannot create KafkConsumer") { + createCustomTopic(topic) + val newProperties = properties + ("BOOTSTRAP_SERVERS" -> "kafka01.internal:9092") + val iter = mockExasolIterator(newProperties, Seq(0), Seq(-1)) + val thrown = intercept[org.apache.kafka.common.KafkaException] { + KafkaMetadata.run(mock[ExaMetadata], iter) + } + assert(thrown.getMessage === "Failed to construct kafka consumer") + } + +} diff --git a/src/main/scala/com/exasol/cloudetl/data/Row.scala b/src/main/scala/com/exasol/cloudetl/data/Row.scala index b755c846..6b74b1a2 100644 --- a/src/main/scala/com/exasol/cloudetl/data/Row.scala +++ b/src/main/scala/com/exasol/cloudetl/data/Row.scala @@ -1,5 +1,7 @@ package com.exasol.cloudetl.data +import java.nio.ByteBuffer + import scala.collection.JavaConverters._ import org.apache.avro.Schema @@ -74,22 +76,34 @@ object Row { case Schema.Type.LONG => value case Schema.Type.FLOAT => value case Schema.Type.DOUBLE => value - case Schema.Type.STRING => value.toString + case Schema.Type.STRING => getAvroValueAsString(value, field) + case Schema.Type.FIXED => getAvroValueAsString(value, field) + case Schema.Type.BYTES => getAvroValueAsString(value, field) case Schema.Type.ENUM => value.toString case Schema.Type.UNION => getAvroUnionValue(value, field) - case Schema.Type.FIXED => value.asInstanceOf[GenericFixed].bytes().toString - case Schema.Type.BYTES => value.asInstanceOf[Utf8].toString case field => - throw new IllegalArgumentException(s"Avro ${field.getName} type is not supperted!") + throw new IllegalArgumentException(s"Avro ${field.getName} type is not supported!") } } + def getAvroValueAsString(value: Any, field: Schema): String = value match { + case str: String => str + case utf: Utf8 => utf.toString + case byteBuffer: ByteBuffer => new String(byteBuffer.array) + case arrayByte: Array[Byte] => new String(arrayByte) + case fixed: GenericFixed => new String(fixed.bytes()) + case other => + throw new IllegalArgumentException( + s"Avro ${field.getName} type with value $other cannot be converted to string!" + ) + } + def getAvroUnionValue(value: Any, field: Schema): Any = field.getTypes.asScala.toSeq match { case Seq(f) => getAvroRecordValue(value, f) case Seq(n, f) if n.getType == Schema.Type.NULL => getAvroRecordValue(value, f) case Seq(f, n) if n.getType == Schema.Type.NULL => getAvroRecordValue(value, f) case _ => - throw new IllegalArgumentException("Avro UNION type should contain a primitive and null!") + throw new IllegalArgumentException("Avro Union type should contain a primitive and null!") } } diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala index c353b6bf..68a63291 100644 --- a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala @@ -247,13 +247,13 @@ object KafkaConsumerProperties extends CommonProperties { * An optional property key name to set SSL secure connections to * Kafka cluster. */ - private[kafka] val SSL_ENABLED: String = "SSL_ENABLED" + private[kafka] final val SSL_ENABLED: String = "SSL_ENABLED" /** * A number of milliseconds to wait for Kafka consumer {@code poll} to * return any data. */ - private[kafka] final val POLL_TIMEOUT_MS = Config[Long]( + private[kafka] final val POLL_TIMEOUT_MS: Config[Long] = Config[Long]( "POLL_TIMEOUT_MS", "", 30000L // scalastyle:ignore magic.number @@ -270,7 +270,7 @@ object KafkaConsumerProperties extends CommonProperties { * * See [[MAX_RECORDS_PER_RUN]]. */ - private[kafka] final val MIN_RECORDS_PER_RUN = Config[Int]( + private[kafka] final val MIN_RECORDS_PER_RUN: Config[Int] = Config[Int]( "MIN_RECORDS_PER_RUN", "", 100 // scalastyle:ignore magic.number @@ -286,7 +286,7 @@ object KafkaConsumerProperties extends CommonProperties { * * See [[MIN_RECORDS_PER_RUN]]. */ - private[kafka] final val MAX_RECORDS_PER_RUN = Config[Int]( + private[kafka] final val MAX_RECORDS_PER_RUN: Config[Int] = Config[Int]( "MAX_RECORDS_PER_RUN", "", 1000000 // scalastyle:ignore magic.number @@ -306,7 +306,7 @@ object KafkaConsumerProperties extends CommonProperties { * by default, since we manage the offset commits ourselves in the * Exasol table. */ - private[kafka] val ENABLE_AUTO_COMMIT = Config[String]( + private[kafka] final val ENABLE_AUTO_COMMIT: Config[String] = Config[String]( "ENABLE_AUTO_COMMIT", ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false" @@ -320,7 +320,7 @@ object KafkaConsumerProperties extends CommonProperties { * * It is a required property that should be provided by the user. */ - private[kafka] val BOOTSTRAP_SERVERS = Config[String]( + private[kafka] final val BOOTSTRAP_SERVERS: Config[String] = Config[String]( "BOOTSTRAP_SERVERS", ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "" @@ -332,7 +332,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is a unique string that identifies the consumer group this * consumer belongs to. */ - private[kafka] val GROUP_ID = Config[String]( + private[kafka] final val GROUP_ID: Config[String] = Config[String]( "GROUP_ID", ConsumerConfig.GROUP_ID_CONFIG, "EXASOL_KAFKA_UDFS_CONSUMERS" @@ -344,7 +344,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is the maximum number of records returned in a single call to * poll() function. Default value is `500`. */ - private[kafka] val MAX_POLL_RECORDS = Config[String]( + private[kafka] final val MAX_POLL_RECORDS: Config[String] = Config[String]( "MAX_POLL_RECORDS", ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "500" @@ -356,7 +356,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is the minimum amount of data the server should return for a * fetch request. Default value is `1`. */ - private[kafka] val FETCH_MIN_BYTES = Config[String]( + private[kafka] final val FETCH_MIN_BYTES: Config[String] = Config[String]( "FETCH_MIN_BYTES", ConsumerConfig.FETCH_MIN_BYTES_CONFIG, "1" @@ -368,7 +368,7 @@ object KafkaConsumerProperties extends CommonProperties { * The Avro value deserializer will be used when user sets this * property value. */ - private[kafka] val SCHEMA_REGISTRY_URL = Config[String]( + private[kafka] final val SCHEMA_REGISTRY_URL: Config[String] = Config[String]( "SCHEMA_REGISTRY_URL", AbstractKafkaAvroSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, "" @@ -381,7 +381,7 @@ object KafkaConsumerProperties extends CommonProperties { * [[SSL_ENABLED]] is set to {@code true}. Default value is * [[SslConfigs.DEFAULT_SSL_PROTOCOL]]. */ - private[kafka] val SECURITY_PROTOCOL = Config[String]( + private[kafka] final val SECURITY_PROTOCOL: Config[String] = Config[String]( "SECURITY_PROTOCOL", CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SslConfigs.DEFAULT_SSL_PROTOCOL @@ -394,7 +394,7 @@ object KafkaConsumerProperties extends CommonProperties { * file. It is required property when [[SSL_ENABLED]] is set to {@code * true}. */ - private[kafka] val SSL_KEY_PASSWORD = Config[String]( + private[kafka] final val SSL_KEY_PASSWORD: Config[String] = Config[String]( "SSL_KEY_PASSWORD", SslConfigs.SSL_KEY_PASSWORD_CONFIG, "" @@ -406,7 +406,7 @@ object KafkaConsumerProperties extends CommonProperties { * It the store password for the keystore file. It is required * property when [[SSL_ENABLED]] is set to {@code true}. */ - private[kafka] val SSL_KEYSTORE_PASSWORD = Config[String]( + private[kafka] final val SSL_KEYSTORE_PASSWORD: Config[String] = Config[String]( "SSL_KEYSTORE_PASSWORD", SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "" @@ -419,7 +419,7 @@ object KafkaConsumerProperties extends CommonProperties { * property when [[SSL_ENABLED]] is set to {@code true} and can be * used for two-way authentication for the clients. */ - private[kafka] val SSL_KEYSTORE_LOCATION = Config[String]( + private[kafka] final val SSL_KEYSTORE_LOCATION: Config[String] = Config[String]( "SSL_KEYSTORE_LOCATION", SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, "" @@ -431,7 +431,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is the password for the truststore file, and required property * when [[SSL_ENABLED]] is set to {@code true}. */ - private[kafka] val SSL_TRUSTSTORE_PASSWORD = Config[String]( + private[kafka] final val SSL_TRUSTSTORE_PASSWORD: Config[String] = Config[String]( "SSL_TRUSTSTORE_PASSWORD", SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "" @@ -443,7 +443,7 @@ object KafkaConsumerProperties extends CommonProperties { * It is the location of the truststore file, and required property * when [[SSL_ENABLED]] is set to {@code true}. */ - private[kafka] val SSL_TRUSTSTORE_LOCATION = Config[String]( + private[kafka] final val SSL_TRUSTSTORE_LOCATION: Config[String] = Config[String]( "SSL_TRUSTSTORE_LOCATION", SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, "" @@ -458,7 +458,7 @@ object KafkaConsumerProperties extends CommonProperties { * is set to {@code true}. Default value is * [[SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM]]. */ - private[kafka] val SSL_ENDPOINT_IDENTIFICATION_ALGORITHM = Config[String]( + private[kafka] final val SSL_ENDPOINT_IDENTIFICATION_ALGORITHM: Config[String] = Config[String]( "SSL_ENDPOINT_IDENTIFICATION_ALGORITHM", SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM diff --git a/src/main/scala/com/exasol/cloudetl/source/ParquetSource.scala b/src/main/scala/com/exasol/cloudetl/source/ParquetSource.scala index cc006b01..d75595b7 100644 --- a/src/main/scala/com/exasol/cloudetl/source/ParquetSource.scala +++ b/src/main/scala/com/exasol/cloudetl/source/ParquetSource.scala @@ -36,15 +36,14 @@ final case class ParquetSource( private[this] def createReader(): ParquetReader[Row] = { val newConf = new Configuration(conf) - getSchema.foreach { schema => - newConf.set(ReadSupport.PARQUET_READ_SCHEMA, schema.toString) - } - try { + getSchema().foreach { schema => + newConf.set(ReadSupport.PARQUET_READ_SCHEMA, schema.toString) + } ParquetReader.builder(new RowReadSupport, path).withConf(newConf).build() } catch { case NonFatal(exception) => - logger.error(s"Could not create parquet reader for path: $path", exception); + logger.error(s"Could not create parquet reader for path: $path", exception) throw exception } } diff --git a/src/test/scala/com/exasol/cloudetl/data/RowTest.scala b/src/test/scala/com/exasol/cloudetl/data/RowTest.scala new file mode 100644 index 00000000..d1743991 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/data/RowTest.scala @@ -0,0 +1,201 @@ +package com.exasol.cloudetl.data + +import scala.collection.JavaConverters._ + +import org.apache.avro.Schema +import org.apache.avro.generic.GenericData +import org.scalatest.FunSuite + +class RowTest extends FunSuite { + + test("fromAvroGenericRecord returns Row from GenericRecord with ByteBuffer column") { + val recordSchema = createRecord( + "record", + createField("col_str", Schema.create(Schema.Type.STRING)) + ) + val record = new GenericData.Record(recordSchema) + record.put("col_str", java.nio.ByteBuffer.wrap(Array[Byte](104, 101, 108, 108, 111))) + assert(Row.fromAvroGenericRecord(record) === Row(Seq("hello"))) + } + + test("fromAvroGenericRecord return Row GenericRecord with single type in Union type") { + val schemaTypes = Schema.createUnion(Schema.create(Schema.Type.LONG)) + val longUnionSchema = createUnion("col_union", schemaTypes) + val recordSchema = createRecord( + "record", + createField("col_str", Schema.create(Schema.Type.STRING)), + longUnionSchema + ) + val record = new GenericData.Record(recordSchema) + record.put("col_str", "123") + record.put("col_union", 13L) + assert(Row.fromAvroGenericRecord(record) === Row(Seq("123", 13L))) + } + + test("fromAvroGenericRecord returns Row from GenericRecord with many columns") { + val fixedSchema = createFixedSchema("fixedSchema", 5) + val enumSchema = createEnumSchema("enumSchema", Seq("A", "B")) + val recordSchema = createRecord( + "record", + createField("col_str", Schema.create(Schema.Type.STRING)), + createPrimitiveUnionField("col_str_union", Schema.Type.STRING), + createField("col_int", Schema.create(Schema.Type.INT)), + createPrimitiveUnionField("col_int_union", Schema.Type.INT), + createField("col_long", Schema.create(Schema.Type.LONG)), + createPrimitiveUnionField("col_long_union", Schema.Type.LONG), + createField("col_double", Schema.create(Schema.Type.DOUBLE)), + createPrimitiveUnionField("col_double_union", Schema.Type.DOUBLE), + createField("col_float", Schema.create(Schema.Type.FLOAT)), + createPrimitiveUnionField("col_float_union", Schema.Type.FLOAT), + createField("col_bool", Schema.create(Schema.Type.BOOLEAN)), + createPrimitiveUnionField("col_bool_union", Schema.Type.BOOLEAN), + createField("col_bytes", Schema.create(Schema.Type.BYTES)), + createPrimitiveUnionField("col_bytes_union", Schema.Type.BYTES), + createField("col_fixed", fixedSchema), + createUnionField("col_fixed_union", fixedSchema), + createField("col_enum", enumSchema), + createUnionField("col_enum_union", enumSchema), + createField("col_null", Schema.create(Schema.Type.NULL)) + ) + + def getFixedData(fixedSchema: Schema, bytes: Array[Byte]): GenericData.Fixed = { + val data = new GenericData.Fixed(fixedSchema) + data.bytes(bytes) + data + } + + val record = new GenericData.Record(recordSchema) + record.put("col_str", "hello") + record.put("col_str_union", "hello") + record.put("col_int", 1) + record.put("col_int_union", 1) + record.put("col_long", 1L) + record.put("col_long_union", 1L) + record.put("col_double", 1.0) + record.put("col_double_union", 1.0) + record.put("col_float", 1.0f) + record.put("col_float_union", 1.0f) + record.put("col_bool", true) + record.put("col_bool_union", false) + record.put("col_bytes", "bytes".getBytes()) + record.put("col_bytes_union", null) + record.put("col_fixed", getFixedData(fixedSchema, Array[Byte](102, 105, 120, 101, 100))) + record.put("col_fixed_union", getFixedData(fixedSchema, Array[Byte](104, 101, 108, 108, 111))) + record.put("col_enum", new GenericData.EnumSymbol(enumSchema, "A")) + record.put("col_enum_union", new GenericData.EnumSymbol(enumSchema, "B")) + record.put("col_null", null) + + val expectedRow = Row( + Seq( + "hello", + "hello", + 1, + 1, + 1L, + 1L, + 1.0, + 1.0, + 1.0f, + 1.0f, + true, + false, + "bytes", + null, + "fixed", + "hello", + "A", + "B", + null + ) + ) + + assert(Row.fromAvroGenericRecord(record) === expectedRow) + } + + test("fromAvroGenericRecord throws if Avro type (nested record) is not supported") { + val innerRecordSchema = createRecord( + "innerRecord", + createPrimitiveUnionField("inner_field", Schema.Type.STRING) + ) + + val recordSchema = createRecord( + "record", + createPrimitiveUnionField("col_str", Schema.Type.STRING), + createField("col_record", innerRecordSchema) + ) + + val innerRecord = new GenericData.Record(innerRecordSchema) + innerRecord.put("inner_field", "abc") + + val record = new GenericData.Record(recordSchema) + record.put("col_str", "xyz") + record.put("col_record", innerRecord) + + val thrown = intercept[IllegalArgumentException] { + Row.fromAvroGenericRecord(record) + } + assert(thrown.getMessage === "Avro record type is not supported!") + } + + test("fromAvroGenericRecord throws if GenericRecord value cannot be cast as string") { + val recordSchema = createRecord( + "record", + createField("col_str", Schema.create(Schema.Type.STRING)) + ) + val record = new GenericData.Record(recordSchema) + record.put("col_str", 1L) + val thrown = intercept[IllegalArgumentException] { + Row.fromAvroGenericRecord(record) + } + assert(thrown.getMessage === "Avro string type with value 1 cannot be converted to string!") + } + + test("fromAvroGenericRecord throws if GenericRecord Union type is not primitive and null") { + val unionSchemaTypes = + Schema.createUnion(Seq(Schema.Type.STRING, Schema.Type.INT).map(Schema.create(_)).asJava) + val unionSchema = createUnion("col_union", unionSchemaTypes) + val recordSchema = createRecord( + "record", + createField("col_str", Schema.create(Schema.Type.STRING)), + unionSchema + ) + val record = new GenericData.Record(recordSchema) + record.put("col_str", "abc") + record.put("col_union", 1) + val thrown = intercept[IllegalArgumentException] { + Row.fromAvroGenericRecord(record) + } + assert(thrown.getMessage === "Avro Union type should contain a primitive and null!") + } + + final def createRecord(name: String, fields: Schema.Field*): Schema = { + val schema = Schema.createRecord(name, name, "com.exasol.cloudetl.row", false) + schema.setFields(fields.asJava) + schema + } + + final def createField(name: String, schema: Schema): Schema.Field = + new Schema.Field(name, schema, "", null, Schema.Field.Order.ASCENDING) + + final def createUnionField(name: String, schemas: Schema*): Schema.Field = { + val schemaTypes = Seq(Schema.create(Schema.Type.NULL)) ++ schemas + val unionSchema = Schema.createUnion(schemaTypes.asJava) + createUnion(name, unionSchema) + } + + final def createPrimitiveUnionField(name: String, types: Schema.Type*): Schema.Field = { + val schemaTypes = (Seq(Schema.Type.NULL) ++ types).map(Schema.create(_)) + val unionSchema = Schema.createUnion(schemaTypes.asJava) + createUnion(name, unionSchema) + } + + final def createUnion(name: String, unionSchema: Schema): Schema.Field = + new Schema.Field(name, unionSchema, null, null, Schema.Field.Order.ASCENDING) + + final def createFixedSchema(name: String, size: Int): Schema = + Schema.createFixed(name, "", "com.exasol.cloudetl.row", size) + + final def createEnumSchema(name: String, ordinals: Seq[String]): Schema = + Schema.createEnum(name, "", "com.exasol.cloudetl.row", ordinals.asJava) + +} diff --git a/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala index 0b140e2c..d1577731 100644 --- a/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala +++ b/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala @@ -1,7 +1,5 @@ package com.exasol.cloudetl.kafka -import org.apache.avro.generic.GenericRecord -import org.apache.kafka.clients.consumer.KafkaConsumer import org.scalatest.BeforeAndAfterEach import org.scalatest.FunSuite @@ -242,20 +240,43 @@ class KafkaConsumerPropertiesTest extends FunSuite with BeforeAndAfterEach { assert(BaseProperties(properties).getSSLEndpointIdentificationAlgorithm() === "https") } - test("build throws if required properties are not provided") { + test("build throws if required BOOTSTRAP_SERVERS property is not provided") { val thrown = intercept[IllegalArgumentException] { BaseProperties(properties).build() } assert(thrown.getMessage === errorMessage("BOOTSTRAP_SERVERS")) } - ignore("build returns a KafkaConsumer[String, GenericRecord]") { - properties = Map( - "BOOTSTRAP_SERVERS" -> "kafka01.internal:9092", - "SCHEMA_REGISTRY_URL" -> "https://schema-registry.internal.com" + test("build throws if required SCHEMA_REGISTRY_URL property is not provided") { + properties = Map("BOOTSTRAP_SERVERS" -> "kafka01.internal:9092") + val thrown = intercept[IllegalArgumentException] { + BaseProperties(properties).build() + } + assert(thrown.getMessage === errorMessage("SCHEMA_REGISTRY_URL")) + } + + test("getProperties returns Java map properties") { + import KafkaConsumerProperties._ + val testData = Map( + BOOTSTRAP_SERVERS -> "kafka.broker.com:9092", + SCHEMA_REGISTRY_URL -> "http://schema-registry.com:8080", + SECURITY_PROTOCOL -> "SSL", + SSL_KEY_PASSWORD -> "sslKeyPass", + SSL_KEYSTORE_PASSWORD -> "sslKeystorePass", + SSL_KEYSTORE_LOCATION -> "/bucket/keystore.JKS", + SSL_TRUSTSTORE_PASSWORD -> "sslTruststorePass", + SSL_TRUSTSTORE_LOCATION -> "/bucket/truststore.JKS" ) - val kafkaConsumer = BaseProperties(properties).build() - assert(kafkaConsumer.isInstanceOf[KafkaConsumer[String, GenericRecord]]) + properties = Map("SSL_ENABLED" -> "true") ++ testData.map { + case (key, value) => + key.userPropertyName -> value + } + val javaProps = BaseProperties(properties).getProperties() + assert(javaProps.isInstanceOf[java.util.Map[String, Object]]) + testData.foreach { + case (key, value) => + assert(javaProps.get(key.kafkaPropertyName) === value) + } } private[this] case class BaseProperties(val params: Map[String, String]) diff --git a/src/test/scala/com/exasol/cloudetl/parquet/ParquetWriteOptionsTest.scala b/src/test/scala/com/exasol/cloudetl/parquet/ParquetWriteOptionsTest.scala new file mode 100644 index 00000000..0aaf94b7 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/parquet/ParquetWriteOptionsTest.scala @@ -0,0 +1,94 @@ +package com.exasol.cloudetl.parquet + +import com.exasol.cloudetl.storage.StorageProperties + +import org.apache.parquet.hadoop.ParquetWriter +import org.apache.parquet.hadoop.metadata.CompressionCodecName +import org.scalatest.BeforeAndAfterEach +import org.scalatest.FunSuite + +class ParquetWriteOptionsTest extends FunSuite with BeforeAndAfterEach { + + private[this] var properties: Map[String, String] = _ + + override final def beforeEach(): Unit = { + properties = Map.empty[String, String] + () + } + + test("apply returns default values") { + val options = ParquetWriteOptions(StorageProperties(properties)) + assert(options.compressionCodec === CompressionCodecName.UNCOMPRESSED) + assert(options.blockSize === ParquetWriter.DEFAULT_BLOCK_SIZE) + assert(options.pageSize === ParquetWriter.DEFAULT_PAGE_SIZE) + assert(options.enableDictionaryEncoding === true) + assert(options.enableValidation === true) + } + + test("apply returns user provided compression codec") { + val testData = Map( + "snappy" -> CompressionCodecName.SNAPPY, + "gzip" -> CompressionCodecName.GZIP, + "lzo" -> CompressionCodecName.LZO, + "other" -> CompressionCodecName.UNCOMPRESSED + ) + testData.foreach { + case (given, expected) => + properties = Map("PARQUET_COMPRESSION_CODEC" -> given) + assert(ParquetWriteOptions(StorageProperties(properties)).compressionCodec === expected) + } + } + + test("apply returns user provided block size") { + properties = Map("PARQUET_BLOCK_SIZE" -> "64") + assert(ParquetWriteOptions(StorageProperties(properties)).blockSize === 64) + } + + test("apply throws if block size value cannot be converted to integer") { + properties = Map("PARQUET_BLOCK_SIZE" -> "6l4") + val thrown = intercept[NumberFormatException] { + ParquetWriteOptions(StorageProperties(properties)) + } + assert(thrown.getMessage === s"""For input string: "6l4"""") + } + + test("apply returns user provided page size") { + properties = Map("PARQUET_PAGE_SIZE" -> "128") + assert(ParquetWriteOptions(StorageProperties(properties)).pageSize === 128) + } + + test("apply throws if page size value cannot be converted to integer") { + properties = Map("PARQUET_PAGE_SIZE" -> "12e") + val thrown = intercept[NumberFormatException] { + ParquetWriteOptions(StorageProperties(properties)) + } + assert(thrown.getMessage === s"""For input string: "12e"""") + } + + test("apply returns user provided dictionary encoding enabled") { + properties = Map("PARQUET_DICTIONARY_ENCODING" -> "false") + assert(ParquetWriteOptions(StorageProperties(properties)).enableDictionaryEncoding === false) + } + + test("apply throws if dictionary encoding enable cannot be converted to boolean") { + properties = Map("PARQUET_DICTIONARY_ENCODING" -> "nay") + val thrown = intercept[IllegalArgumentException] { + ParquetWriteOptions(StorageProperties(properties)) + } + assert(thrown.getMessage === s"""For input string: "nay"""") + } + + test("apply returns user provided validation enabled") { + properties = Map("PARQUET_VALIDATION" -> "false") + assert(ParquetWriteOptions(StorageProperties(properties)).enableValidation === false) + } + + test("apply throws if validation enable cannot be converted to boolean") { + properties = Map("PARQUET_VALIDATION" -> "yay") + val thrown = intercept[IllegalArgumentException] { + ParquetWriteOptions(StorageProperties(properties)) + } + assert(thrown.getMessage === s"""For input string: "yay"""") + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/KafkaPathTest.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/KafkaPathTest.scala new file mode 100644 index 00000000..0d922cf5 --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/KafkaPathTest.scala @@ -0,0 +1,82 @@ +package com.exasol.cloudetl.scriptclasses + +import scala.collection.JavaConverters._ + +import com.exasol.cloudetl.kafka.KafkaConsumerProperties + +import org.mockito.Mockito._ + +class KafkaPathTest extends PathTest { + + private[this] val kafkaConsumerProperties = Map( + "BOOTSTRAP_SERVERS" -> "kafka.broker01.example.com:9092", + "TOPICS" -> "kafkaTopic", + "TABLE_NAME" -> "exasolTable" + ) + + override final def beforeEach(): Unit = { + super.beforeEach() + properties = kafkaConsumerProperties + when(metadata.getScriptSchema()).thenReturn(schema) + () + } + + test("generateSqlForImportSpec returns SQL statement") { + when(importSpec.getParameters()).thenReturn(properties.asJava) + val propertyPairs = KafkaConsumerProperties(properties).mkString() + + val expectedSQLStatement = + s"""SELECT + | $schema.KAFKA_IMPORT( + | '$propertyPairs', partition_index, max_offset + |) + |FROM ( + | SELECT $schema.KAFKA_METADATA( + | '$propertyPairs', kafka_partition, kafka_offset + | ) FROM ( + | SELECT kafka_partition, MAX(kafka_offset) AS kafka_offset + | FROM exasolTable + | GROUP BY kafka_partition + | UNION ALL + | SELECT 0, -1 + | FROM DUAL + | WHERE NOT EXISTS (SELECT * FROM exasolTable LIMIT 2) + | ) + |) + |GROUP BY + | partition_index; + |""".stripMargin + + assert(KafkaPath.generateSqlForImportSpec(metadata, importSpec) === expectedSQLStatement) + verify(metadata, atLeastOnce).getScriptSchema + verify(importSpec, times(1)).getParameters + } + + test("generateSqlForImportSpec throws if table name property is not set") { + properties -= ("TABLE_NAME") + when(importSpec.getParameters()).thenReturn(properties.asJava) + val thrown = intercept[IllegalArgumentException] { + KafkaPath.generateSqlForImportSpec(metadata, importSpec) + } + assert(thrown.getMessage === "Please provide a value for the TABLE_NAME property!") + } + + test("generateSqlForImportSpec throws if topics property is not set") { + properties -= "TOPICS" + when(importSpec.getParameters()).thenReturn(properties.asJava) + val thrown = intercept[IllegalArgumentException] { + KafkaPath.generateSqlForImportSpec(metadata, importSpec) + } + assert(thrown.getMessage === "Please provide a value for the TOPICS property!") + } + + test("generateSqlForImportSpec throws if topics contains more than one topic") { + properties += ("TOPICS" -> "topic1,topic2,topic3") + when(importSpec.getParameters()).thenReturn(properties.asJava) + val thrown = intercept[IllegalArgumentException] { + KafkaPath.generateSqlForImportSpec(metadata, importSpec) + } + assert(thrown.getMessage === "Only single topic can be consumed using Kafka import!") + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/scriptclasses/PathTest.scala b/src/test/scala/com/exasol/cloudetl/scriptclasses/PathTest.scala index 61f34fbc..f48251e4 100644 --- a/src/test/scala/com/exasol/cloudetl/scriptclasses/PathTest.scala +++ b/src/test/scala/com/exasol/cloudetl/scriptclasses/PathTest.scala @@ -15,7 +15,7 @@ trait PathTest extends FunSuite with BeforeAndAfterEach with MockitoSugar { private[scriptclasses] val schema = "myDBSchema" - private[scriptclasses] val properties = Map( + private[scriptclasses] var properties = Map( "BUCKET_PATH" -> "s3a://my_bucket/folder1/*", "DATA_FORMAT" -> "PARQUET", "S3_ENDPOINT" -> "s3.eu-central-1.com", @@ -27,7 +27,8 @@ trait PathTest extends FunSuite with BeforeAndAfterEach with MockitoSugar { private[scriptclasses] var importSpec: ExaImportSpecification = _ private[scriptclasses] var exportSpec: ExaExportSpecification = _ - override final def beforeEach(): Unit = { + @SuppressWarnings(Array("org.wartremover.contrib.warts.UnsafeInheritance")) + override def beforeEach(): Unit = { metadata = mock[ExaMetadata] importSpec = mock[ExaImportSpecification] exportSpec = mock[ExaExportSpecification] diff --git a/src/test/scala/com/exasol/cloudetl/source/AbstractSourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/AbstractSourceTest.scala new file mode 100644 index 00000000..4f505d8a --- /dev/null +++ b/src/test/scala/com/exasol/cloudetl/source/AbstractSourceTest.scala @@ -0,0 +1,50 @@ +package com.exasol.cloudetl.source + +import java.nio.file.Path +import java.nio.file.Paths + +import com.exasol.cloudetl.storage.FileFormat +import com.exasol.cloudetl.util.FileSystemUtil + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.scalatest.BeforeAndAfterEach +import org.scalatest.FunSuite + +@SuppressWarnings(Array("org.wartremover.warts.Overloading")) +class AbstractSourceTest extends FunSuite with BeforeAndAfterEach { + + private[this] var conf: Configuration = _ + private[this] var fileSystem: FileSystem = _ + + private[source] val format: String = "dummy" + private[source] var resourceDir: Path = _ + + override final def beforeEach(): Unit = { + conf = new Configuration() + fileSystem = FileSystem.get(conf) + resourceDir = Paths.get(getClass.getResource(s"/data/import/$format").toURI).toAbsolutePath + () + } + + final def getConf(): Configuration = conf + + final def getFileSystem(): FileSystem = fileSystem + + final def getSource(filePath: org.apache.hadoop.fs.Path): Source = + Source(FileFormat(format), filePath, conf, fileSystem) + + final def getSource(filePath: org.apache.hadoop.fs.Path, fileFormat: String): Source = + Source(FileFormat(fileFormat), filePath, conf, fileSystem) + + final def getRecordsCount(filePath: Path): Int = { + val globbedFilePath = FileSystemUtil.globWithLocal(filePath, fileSystem) + globbedFilePath.map { file => + val src = getSource(file) + val cnt = src.stream().size + src.close() + cnt + }.sum + } + +} diff --git a/src/test/scala/com/exasol/cloudetl/source/AvroSourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/AvroSourceTest.scala index 2c9f7f86..554843ed 100644 --- a/src/test/scala/com/exasol/cloudetl/source/AvroSourceTest.scala +++ b/src/test/scala/com/exasol/cloudetl/source/AvroSourceTest.scala @@ -2,7 +2,7 @@ package com.exasol.cloudetl.source import java.nio.file.Paths -class AvroSourceTest extends SourceTest { +class AvroSourceTest extends AbstractSourceTest { override val format: String = "avro" @@ -11,4 +11,12 @@ class AvroSourceTest extends SourceTest { assert(getRecordsCount(filePath) === 1998) } + test("stream throws if it cannot create AVRO reader") { + val nonPath = new org.apache.hadoop.fs.Path(s"$resourceDir/notFile.avro") + val thrown = intercept[java.io.FileNotFoundException] { + getSource(nonPath).stream().size + } + assert(thrown.getMessage === s"File $nonPath does not exist") + } + } diff --git a/src/test/scala/com/exasol/cloudetl/source/OrcSourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/OrcSourceTest.scala index 89ad8957..e30c93a0 100644 --- a/src/test/scala/com/exasol/cloudetl/source/OrcSourceTest.scala +++ b/src/test/scala/com/exasol/cloudetl/source/OrcSourceTest.scala @@ -2,7 +2,7 @@ package com.exasol.cloudetl.source import java.nio.file.Paths -class OrcSourceTest extends SourceTest { +class OrcSourceTest extends AbstractSourceTest { override val format: String = "orc" @@ -16,4 +16,12 @@ class OrcSourceTest extends SourceTest { assert(getRecordsCount(filePath) === 438304) } + test("stream throws if it cannot create ORC reader") { + val nonPath = new org.apache.hadoop.fs.Path(s"$resourceDir/notFile.orc") + val thrown = intercept[java.io.FileNotFoundException] { + getSource(nonPath).stream().size + } + assert(thrown.getMessage === s"File $nonPath does not exist") + } + } diff --git a/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala index 0e5f7167..9c82fc2d 100644 --- a/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala +++ b/src/test/scala/com/exasol/cloudetl/source/ParquetSourceTest.scala @@ -6,7 +6,7 @@ import com.exasol.cloudetl.util.FileSystemUtil import org.apache.parquet.schema.MessageTypeParser -class ParquetSourceTest extends SourceTest { +class ParquetSourceTest extends AbstractSourceTest { override val format: String = "parquet" @@ -47,4 +47,12 @@ class ParquetSourceTest extends SourceTest { assert(getRecordsCount(filePath) === 999) } + test("stream throws if it cannot create PARQUET reader") { + val nonPath = new org.apache.hadoop.fs.Path(s"$resourceDir/notFile.parquet") + val thrown = intercept[java.io.FileNotFoundException] { + getSource(nonPath).stream().size + } + assert(thrown.getMessage === s"File $nonPath does not exist") + } + } diff --git a/src/test/scala/com/exasol/cloudetl/source/SourceTest.scala b/src/test/scala/com/exasol/cloudetl/source/SourceTest.scala index 8848dabd..8eca423f 100644 --- a/src/test/scala/com/exasol/cloudetl/source/SourceTest.scala +++ b/src/test/scala/com/exasol/cloudetl/source/SourceTest.scala @@ -1,43 +1,15 @@ package com.exasol.cloudetl.source -import java.nio.file.Path -import java.nio.file.Paths +class SourceTest extends AbstractSourceTest { -import com.exasol.cloudetl.storage.FileFormat -import com.exasol.cloudetl.util.FileSystemUtil + override val format: String = "avro" -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.scalatest.BeforeAndAfterEach -import org.scalatest.FunSuite - -class SourceTest extends FunSuite with BeforeAndAfterEach { - - private[this] var conf: Configuration = _ - private[this] var fileSystem: FileSystem = _ - - private[source] val format: String = "dummy" - private[source] var resourceDir: Path = _ - - override final def beforeEach(): Unit = { - conf = new Configuration() - fileSystem = FileSystem.get(conf) - resourceDir = Paths.get(getClass.getResource(s"/data/import/$format").toURI).toAbsolutePath - () - } - - final def getConf(): Configuration = conf - - final def getFileSystem(): FileSystem = fileSystem - - final def getRecordsCount(filePath: Path): Int = { - val globbedFilePath = FileSystemUtil.globWithLocal(filePath, fileSystem) - globbedFilePath.map { file => - val source = Source(FileFormat(format), file, conf, fileSystem) - val cnt = source.stream().size - source.close() - cnt - }.sum + test("apply throws if file format is not supported") { + val path = new org.apache.hadoop.fs.Path(s"$resourceDir/sales10.avro") + val thrown = intercept[IllegalArgumentException] { + getSource(path, "file") + } + assert(thrown.getMessage === "Unsupported storage format: 'FILE'") } } From 91b90ff286ca25d148f63ab3b328ddd94dbdd7cb Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Fri, 25 Oct 2019 18:53:32 +0200 Subject: [PATCH 16/18] Add two additional Kafka consumer properties. Adds these Kafka consumer parameters that can provided to configure the UDF Kafka consumer: - fetch.max.bytes - max.partition.fetch.bytes --- .../kafka/KafkaConsumerProperties.scala | 44 +++++++++++++++++++ .../cloudetl/scriptclasses/KafkaImport.scala | 2 +- .../kafka/KafkaConsumerPropertiesTest.scala | 34 ++++++++++++-- 3 files changed, 76 insertions(+), 4 deletions(-) diff --git a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala index 68a63291..d45baf98 100644 --- a/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala +++ b/src/main/scala/com/exasol/cloudetl/kafka/KafkaConsumerProperties.scala @@ -107,6 +107,21 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) final def getFetchMinBytes(): String = get(FETCH_MIN_BYTES.userPropertyName).fold(FETCH_MIN_BYTES.defaultValue)(identity) + /** + * Returns {@code FETCH_MAX_BYTES} property value if provided, + * otherwise returns the default value. + */ + final def getFetchMaxBytes(): String = + get(FETCH_MAX_BYTES.userPropertyName).fold(FETCH_MAX_BYTES.defaultValue)(identity) + + /** + * Returns {@code MAX_PARTITION_FETCH_BYTES} property value if + * provided, otherwise returns the default value. + */ + final def getMaxPartitionFetchBytes(): String = + get(MAX_PARTITION_FETCH_BYTES.userPropertyName) + .fold(MAX_PARTITION_FETCH_BYTES.defaultValue)(identity) + // Secure Connection Related Properties /** @@ -185,6 +200,8 @@ class KafkaConsumerProperties(private val properties: Map[String, String]) props.put(SCHEMA_REGISTRY_URL.kafkaPropertyName, getSchemaRegistryUrl()) props.put(MAX_POLL_RECORDS.kafkaPropertyName, getMaxPollRecords()) props.put(FETCH_MIN_BYTES.kafkaPropertyName, getFetchMinBytes()) + props.put(FETCH_MAX_BYTES.kafkaPropertyName, getFetchMaxBytes()) + props.put(MAX_PARTITION_FETCH_BYTES.kafkaPropertyName, getMaxPartitionFetchBytes()) if (isSSLEnabled()) { props.put(SECURITY_PROTOCOL.kafkaPropertyName, getSecurityProtocol()) props.put(SSL_KEY_PASSWORD.kafkaPropertyName, getSSLKeyPassword()) @@ -362,6 +379,33 @@ object KafkaConsumerProperties extends CommonProperties { "1" ) + /** + * This is the {@code fetch.max.bytes} configuration setting. + * + * It is the maximum amount of data the server should return for a + * fetch request. Default value is + * [[ConsumerConfig.DEFAULT_FETCH_MAX_BYTES]]. + */ + private[kafka] final val FETCH_MAX_BYTES: Config[String] = Config[String]( + "FETCH_MAX_BYTES", + ConsumerConfig.FETCH_MAX_BYTES_CONFIG, + s"${ConsumerConfig.DEFAULT_FETCH_MAX_BYTES}" + ) + + /** + * This is the {@code max.partition.fetch.bytes} configuration + * setting. + * + * It is the maximum amount of data the server will return per + * partition. Default value is + * [[ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES]]. + */ + private[kafka] final val MAX_PARTITION_FETCH_BYTES: Config[String] = Config[String]( + "MAX_PARTITION_FETCH_BYTES", + ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, + s"${ConsumerConfig.DEFAULT_MAX_PARTITION_FETCH_BYTES}" + ) + /** * An optional schema registry url. * diff --git a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala index 3d05fec2..91806e52 100644 --- a/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala +++ b/src/main/scala/com/exasol/cloudetl/scriptclasses/KafkaImport.scala @@ -61,7 +61,7 @@ object KafkaImport extends LazyLogging { iterator.emit(allColumns: _*) } logger.info( - s"Emitted total=$recordsCount records in node=$nodeId, vm=$vmId, partition=$partitionId" + s"Emitted total=$total records in node=$nodeId, vm=$vmId, partition=$partitionId" ) } while (recordsCount >= minRecords && total < maxRecords) diff --git a/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala b/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala index d1577731..cec51f93 100644 --- a/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala +++ b/src/test/scala/com/exasol/cloudetl/kafka/KafkaConsumerPropertiesTest.scala @@ -158,6 +158,25 @@ class KafkaConsumerPropertiesTest extends FunSuite with BeforeAndAfterEach { assert(BaseProperties(properties).getFetchMinBytes() === "1") } + test("getFetchMaxBytes returns maximum fetch bytes property value") { + properties = Map("FETCH_MAX_BYTES" -> "27") + assert(BaseProperties(properties).getFetchMaxBytes() === "27") + } + + test("getFetchMaxBytes returns default value if property is not set") { + // intentionally hardcoded, get alert if it changes. + assert(BaseProperties(properties).getFetchMaxBytes() === "52428800") + } + + test("getMaxPartitionFetchBytes returns maximum partition fetch bytes property value") { + properties = Map("MAX_PARTITION_FETCH_BYTES" -> "4") + assert(BaseProperties(properties).getMaxPartitionFetchBytes() === "4") + } + + test("getMaxPartitionFetchBytes returns default value if property is not set") { + assert(BaseProperties(properties).getMaxPartitionFetchBytes() === "1048576") + } + test("getSecurityProtocol returns user provided security protocol property value") { properties = Map("SECURITY_PROTOCOL" -> "SSL") assert(BaseProperties(properties).getSecurityProtocol() === "SSL") @@ -257,7 +276,7 @@ class KafkaConsumerPropertiesTest extends FunSuite with BeforeAndAfterEach { test("getProperties returns Java map properties") { import KafkaConsumerProperties._ - val testData = Map( + val requiredProperties = Map( BOOTSTRAP_SERVERS -> "kafka.broker.com:9092", SCHEMA_REGISTRY_URL -> "http://schema-registry.com:8080", SECURITY_PROTOCOL -> "SSL", @@ -267,13 +286,22 @@ class KafkaConsumerPropertiesTest extends FunSuite with BeforeAndAfterEach { SSL_TRUSTSTORE_PASSWORD -> "sslTruststorePass", SSL_TRUSTSTORE_LOCATION -> "/bucket/truststore.JKS" ) - properties = Map("SSL_ENABLED" -> "true") ++ testData.map { + val optionalProperties = Map( + ENABLE_AUTO_COMMIT -> "false", + GROUP_ID -> "EXASOL_KAFKA_UDFS_CONSUMERS", + MAX_POLL_RECORDS -> "500", + FETCH_MIN_BYTES -> "1", + FETCH_MAX_BYTES -> "52428800", + MAX_PARTITION_FETCH_BYTES -> "1048576" + ) + + properties = Map("SSL_ENABLED" -> "true") ++ requiredProperties.map { case (key, value) => key.userPropertyName -> value } val javaProps = BaseProperties(properties).getProperties() assert(javaProps.isInstanceOf[java.util.Map[String, Object]]) - testData.foreach { + (requiredProperties ++ optionalProperties).foreach { case (key, value) => assert(javaProps.get(key.kafkaPropertyName) === value) } From e692887c0d39132ba26f48cc3e60bd75d2127750 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Sun, 27 Oct 2019 12:07:48 +0100 Subject: [PATCH 17/18] Fix the Scala versions mismatch issue with integration test embeddedkafka schema registry dependency. This was strange error that occurs when running on Scala 2.11.12 version but not on Scala 2.12.8 version. The error: ``` [error] Uncaught exception when running com.exasol.cloudetl.scriptclasses.KafkaMetadataIT: java.lang.NoSuchMethodError: scala.Predef$.refArrayOps([Ljava/lang/Object;)[Ljava/lang/Object; [error] sbt.ForkMain$ForkError: java.lang.NoSuchMethodError: scala.Predef$.refArrayOps([Ljava/lang/Object;)[Ljava/lang/Object; [error] at kafka.cluster.EndPoint$.(EndPoint.scala:32) [error] at kafka.cluster.EndPoint$.(EndPoint.scala) [error] at kafka.server.Defaults$.(KafkaConfig.scala:69) [error] at kafka.server.Defaults$.(KafkaConfig.scala) [error] at kafka.server.KafkaConfig$.(KafkaConfig.scala:854) [error] at kafka.server.KafkaConfig$.(KafkaConfig.scala) [error] at net.manub.embeddedkafka.ops.KafkaOps$class.startKafka(kafkaOps.scala:35) [error] at net.manub.embeddedkafka.schemaregistry.EmbeddedKafka$.startKafka(EmbeddedKafka.scala:62) [error] at net.manub.embeddedkafka.ops.KafkaOps$class.startKafka(kafkaOps.scala:58) [error] at net.manub.embeddedkafka.schemaregistry.EmbeddedKafka$.startKafka(EmbeddedKafka.scala:62) [error] at net.manub.embeddedkafka.schemaregistry.EmbeddedKafka$.start(EmbeddedKafka.scala:85) [error] at com.exasol.cloudetl.kafka.KafkaIntegrationTest$class.beforeEach(KafkaIntegrationTest.scala:30) [error] at com.exasol.cloudetl.scriptclasses.KafkaMetadataIT.beforeEach(KafkaMetadataIT.scala:16) ``` My initial gut feeling was that dependency versions do not match. That is, when I run it with `./sbtx ++2.11.12 it:test` it downloads the Scala `2.11` version of embedded-kafka-schema-registry but not the Scala `2.11` version of org-apache-kafka dependency. I checked into embedded-kafka dependencies, they look okay to me. However, I still do not understand why this issue occurs. My solution was to explicitly define the kafka dependency and remove the one coming with the embedded-kafka-schema-registry. Updated dependencies: ```scala private val TestDependencies: Seq[ModuleID] = Seq( ... "org.apache.kafka" %% "kafka" % "2.3.0" exclude ("org.slf4j", "slf4j-log4j12") exclude ("org.apache.kafka", "kafka-clients"), "io.github.embeddedkafka" %% "embedded-kafka-schema-registry" % "5.3.0" exclude ("org.apacha.kafka", "kafka") ).map(_ % Test) ``` --- project/Dependencies.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index cea9cd90..6b1a7428 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -44,7 +44,8 @@ object Dependencies { private val TestDependencies: Seq[ModuleID] = Seq( "org.scalatest" %% "scalatest" % "3.0.5", "org.mockito" % "mockito-core" % "2.23.4", - "io.github.embeddedkafka" %% "embedded-kafka-schema-registry" % "5.3.0" + "org.apache.kafka" %% "kafka" % "2.3.0" exclude ("org.slf4j", "slf4j-log4j12") exclude ("org.apache.kafka", "kafka-clients"), + "io.github.embeddedkafka" %% "embedded-kafka-schema-registry" % "5.3.0" exclude ("org.apacha.kafka", "kafka") ).map(_ % Test) lazy val ExcludedDependencies: Seq[InclExclRule] = Seq( From fcd6b20205865609d2754404cf47166254b96736 Mon Sep 17 00:00:00 2001 From: Muhammet Orazov Date: Thu, 31 Oct 2019 08:50:44 +0100 Subject: [PATCH 18/18] Update docs to include new changes (#47) This is squashed merged with several commits: * Adds AUTHORS.md file. We received a patch from Hari Nair (CommScope), since he is not visible in the Github contributors lists, we would like to give him credit with authors document. * Updates the documentation with latest changes. Reworked the documentation to incorporate the new Kafka import feature. Similarly, added several new descriptions on user provided properties. * Adds developer guide documentation. This shows general development workflow of the project. It provides instructions on how to perform build, test and release actions. * Adds coveralls code coverage plugin. So far we use the codecov plugin, but it is usually flakey. This coveralls addition should show additional code coverage results. --- .travis.yml | 1 + CONTRIBUTING.md | 2 +- README.md | 264 +++-------------------- docs/deployment_guide.md | 156 ++++++++++++++ docs/developer_guide.md | 166 +++++++++++++++ docs/images/code_coverage_example.png | Bin 0 -> 163096 bytes docs/kafka/import.md | 148 +++++++++++++ docs/overview.md | 193 ----------------- docs/storage/cloud_storages.md | 295 ++++++++++++++++++++++++++ docs/user_guide.md | 282 ++++++++++++++++++++++++ project/Settings.scala | 2 +- project/plugins.sbt | 14 +- 12 files changed, 1093 insertions(+), 430 deletions(-) create mode 100644 docs/deployment_guide.md create mode 100644 docs/developer_guide.md create mode 100644 docs/images/code_coverage_example.png create mode 100644 docs/kafka/import.md delete mode 100644 docs/overview.md create mode 100644 docs/storage/cloud_storages.md create mode 100644 docs/user_guide.md diff --git a/.travis.yml b/.travis.yml index 840f1200..ef953fb1 100644 --- a/.travis.yml +++ b/.travis.yml @@ -37,6 +37,7 @@ script: after_success: - bash <(curl -s https://codecov.io/bash) + - ./sbtx coveralls before_deploy: - echo "Ensure assembly jar file is created for a $TRAVIS_TAG" diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index ed2f73b0..bb709dfe 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -64,7 +64,7 @@ steps below to submit your patches. - **Make sure everything is working**, run `./scripts/ci.sh` - If everything is okay, commit and push to your fork - [Submit a pull request][submit-pr] -- Let's work together to get your changes reviewed +- Let us work together to get your changes reviewed - Merge into master or development branches If your commit fixes any particular issue, please specify it in your commit diff --git a/README.md b/README.md index 9ad9e09f..e90f38b3 100644 --- a/README.md +++ b/README.md @@ -2,6 +2,7 @@ [![Build Status][travis-badge]][travis-link] [![Codecov][codecov-badge]][codecov-link] +[![Coveralls][coveralls-badge]][coveralls-link] [![GitHub Latest Release][gh-release-badge]][gh-release-link]

@@ -10,239 +11,37 @@ source project which is officially supported by Exasol. For any question, you can contact our support team.

-## Table of contents - -* [Overview](#overview) -* [A short example](#a-short-example) -* [Features](#features) -* [Configuration](#configuration-parameters) -* [Setup and deployment](#setup-and-deployment) -* [Building from source](#building-from-source) -* [Contributing](#contributing) - ## Overview -This repository contains helper code to create [Exasol][exasol] ETL UDFs in -order to read from and write to public cloud storage services such as [AWS -S3][s3], [Google Cloud Storage][gcs] and [Azure Blob Storage][azure]. - -Please be aware that Exasol already supports natively [loading CSV format from -AWS S3][sol-594]; however, not from Google Cloud Storage and Azure Storage -systems. Additionally, transfering data between Exasol and [Apache -Hive][apache-hive] is supported via [Hadoop ETL UDFs][hadoop-etl-udfs]. - -## A short example - -Here we show an excerpt from a simple example of importing and exporting Parquet -formatted data stored in Amazon S3. - -Please see [the full list of all cloud storage providers and guidelines to -configure them](./docs/overview.md). - -### Create an Exasol table - -We are going to use a `SALES_POSITIONS` Exasol table to import data into or to -export its contents to Amazon S3. - -```sql -CREATE SCHEMA RETAIL; -OPEN SCHEMA RETAIL; - -DROP TABLE IF EXISTS SALES_POSITIONS; - -CREATE TABLE SALES_POSITIONS ( - SALES_ID INTEGER, - POSITION_ID SMALLINT, - ARTICLE_ID SMALLINT, - AMOUNT SMALLINT, - PRICE DECIMAL(9,2), - VOUCHER_ID SMALLINT, - CANCELED BOOLEAN -); -``` - -### Import from S3 - -```sql -IMPORT INTO SALES_POSITIONS -FROM SCRIPT ETL.IMPORT_PATH WITH - BUCKET_PATH = 's3a://my-bucket/parquet/import/sales_positions/*' - DATA_FORMAT = 'PARQUET' - S3_ACCESS_KEY = 'MY_AWS_ACCESS_KEY' - S3_SECRET_KEY = 'MY_AWS_SECRET_KEY' - S3_ENDPOINT = 's3.MY_REGION.amazonaws.com' - PARALLELISM = 'nproc()'; -``` +This repository contains helper code to create [Exasol][exasol] user defined +functions (UDFs) in order to read from and write to public cloud storage +systems. -### Export to S3 - -```sql -EXPORT SALES_POSITIONS -INTO SCRIPT ETL.EXPORT_PATH WITH - BUCKET_PATH = 's3a://my-bucket/parquet/export/sales_positions/' - S3_ACCESS_KEY = 'MY_AWS_ACCESS_KEY' - S3_SECRET_KEY = 'MY_AWS_SECRET_KEY' - S3_ENDPOINT = 's3.MY_REGION.amazonaws.com' - PARALLELISM = 'iproc(), floor(random()*4)'; -``` - -Please change the paths and parameters accordingly. +Additionally, it provides UDF scripts to import data from [Apache +Kafka][apache-kafka] clusters. ## Features -The following table shows currently supported features with the latest realese. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
Storage System / Data FormatParquetAvroOrc
IMPORTEXPORTIMPORTEXPORTIMPORTEXPORT
Amazon S3
Google Cloud Storage
Azure Blob Storage
Azure Data Lake (Gen1) Storage
- -## Configuration Parameters - -The following configuration parameters should be provided when using the -cloud-storage-etl-udfs. - -| Parameter | Default | Description -|:-------------------------------|:---------------|:---------------------------------------------------------------------------------------------------------------------------------------------------------| -|``BUCKET_PATH`` |** |A path to the data bucket. It should start with cloud storage system specific schema, for example `s3a`. | -|``DATA_FORMAT`` |``PARQUET`` |The data storage format in the provided path. | -|``PARALLELISM IN IMPORT`` |``nproc()`` |The number of parallel instances to be started for importing data. *Please multiply this to increase the parallelism*. | -|``PARALLELISM IN EXPORT`` |``iproc()`` |The parallel instances for exporting data. *Add another random number to increase the parallelism per node*. For example, ``iproc(), floor(random()*4)``. | -|``PARQUET_COMPRESSION_CODEC`` |``uncompressed``|The compression codec to use when exporting the data into parquet files. Other options are: `snappy`, `gzip` and `lzo`. | -|``EXPORT_BATCH_SIZE`` |``100000`` |The number of records per file from each vm. For exampl, if a single vm gets `1M` records, it will export ten files with default 100000 records each. | -|``storage specific parameters`` |** |These are parameters for specific cloud storage for authentication purpose. | - -Please see [the parameters specific for each cloud storage and how to configure -them here](./docs/overview.md). - -## Setup and deployment - -Please follow the steps described below in order to setup the `IMPORT` and -`EXPORT` UDF scripts. - -### Download the file - -Download the latest jar file from [releases][jars]. - -Additionally, you can also build it from the source by following the [build from -source](#building-from-source) guide. This will allow you to use latest commits -that are not released yet. +* Import formatted data from public cloud storage systems. +* Following data formats are supported as source file format when importing: + [Apache Avro][avro], [Apache Orc][orc] and [Apache Parquet][parquet]. +* Export Exasol table data to public cloud storage systems. +* Following data formats are supported as sink file format when exporting: + [Apache Parquet][parquet]. +* Following cloud storage systems are supported: [Amazon S3][s3], [Google Cloud + Storage][gcs], [Azure Blob Storage][azure-blob] and [Azure Data Lake (Gen1) + Storage][azure-data-lake]. +* Import Apache Avro formatted data from Apache Kafka clusters. -### Create Exasol Bucket +## Documentation -In order to use the import or export functionality of `cloud-storage-etl-udfs`, -you have to upload the jar to a bucket in the Exasol bucket file system -(BucketFS). +For more information please check out the following guides. -For this overview we are using an example bucket named `bucket1`. - -### Upload the JAR file to the bucket - -This will allow using the jar in the ETL UDF scripts later on. Before uploading -the jar, please make sure that the BucketFS ports are open. - -Here we use the port number `2580` for http. - -```bash -curl \ - -X PUT \ - -T path/to/jar/cloud-storage-etl-udfs-{VERSION}.jar \ - http://w:MY-PASSWORD@EXA-NODE-ID:2580/bucket1/cloud-storage-etl-udfs-{VERSION}.jar -``` - -Please change other required parameters such as `VERSION`, `EXA-NODE-ID`. - -### Create ETL UDFs scripts - -Run the following SQL commands to create Exasol scripts. - -```sql -CREATE SCHEMA ETL; -OPEN SCHEMA ETL; - --- Import related scripts - -CREATE OR REPLACE JAVA SET SCRIPT IMPORT_PATH(...) EMITS (...) AS -%scriptclass com.exasol.cloudetl.scriptclasses.ImportPath; -%jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-{VERSION}.jar; -/ - -CREATE OR REPLACE JAVA SET SCRIPT IMPORT_FILES(...) EMITS (...) AS -%env LD_LIBRARY_PATH=/tmp/; -%scriptclass com.exasol.cloudetl.scriptclasses.ImportFiles; -%jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-{VERSION}.jar; -/ - -CREATE OR REPLACE JAVA SCALAR SCRIPT IMPORT_METADATA(...) -EMITS (filename VARCHAR(200), partition_index VARCHAR(100)) AS -%scriptclass com.exasol.cloudetl.scriptclasses.ImportMetadata; -%jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-{VERSION}.jar; -/ - --- Export related scripts - -CREATE OR REPLACE JAVA SET SCRIPT EXPORT_PATH(...) EMITS (...) AS -%scriptclass com.exasol.cloudetl.scriptclasses.ExportPath; -%jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-{VERSION}.jar; -/ - -CREATE OR REPLACE JAVA SET SCRIPT EXPORT_TABLE(...) EMITS (ROWS_AFFECTED INT) AS -%scriptclass com.exasol.cloudetl.scriptclasses.ExportTable; -%jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-{VERSION}.jar; -/ -``` - -Please do not forget to change the bucket name or the latest jar version -according to your setup. - -## Building from source - -Clone the repository, - -```bash -git clone https://github.com/exasol/cloud-storage-etl-udfs - -cd cloud-storage-etl-udfs/ -``` - -Create assembly jar, - -```bash -./sbtx assembly -``` - -The packaged jar should be located at -`target/scala-2.12/cloud-storage-etl-udfs-{VERSION}.jar`. +* [User Guide](docs/user_guide.md) + - [Cloud Storage Systems](docs/storage/cloud_storages.md) + - [Apache Kafka Import](docs/kafka/import.md) +* [Deployment Guide](docs/deployment_guide.md) +* [Developer Guide](docs/developer_guide.md) ## Contributing @@ -251,20 +50,21 @@ See [CONTRIBUTING.md](CONTRIBUTING.md) for contribution guidelines. For requesting a feature, providing a feedback or reporting an issue, please open a [Github issue][gh-issues]. -[travis-badge]: https://travis-ci.org/exasol/cloud-storage-etl-udfs.svg?branch=master +[travis-badge]: https://img.shields.io/travis/exasol/cloud-storage-etl-udfs/master.svg?logo=travis [travis-link]: https://travis-ci.org/exasol/cloud-storage-etl-udfs [codecov-badge]: https://codecov.io/gh/exasol/cloud-storage-etl-udfs/branch/master/graph/badge.svg [codecov-link]: https://codecov.io/gh/exasol/cloud-storage-etl-udfs -[gh-release-badge]: https://img.shields.io/github/release/exasol/cloud-storage-etl-udfs.svg +[coveralls-badge]: https://img.shields.io/coveralls/exasol/cloud-storage-etl-udfs.svg +[coveralls-link]: https://coveralls.io/github/exasol/cloud-storage-etl-udfs +[gh-release-badge]: https://img.shields.io/github/release/exasol/cloud-storage-etl-udfs.svg?logo=github [gh-release-link]: https://github.com/exasol/cloud-storage-etl-udfs/releases/latest [gh-issues]: https://github.com/exasol/cloud-storage-etl-udfs/issues [exasol]: https://www.exasol.com/en/ -[sol-594]: https://www.exasol.com/support/browse/SOL-594 -[apache-hive]: https://hive.apache.org/ -[hadoop-etl-udfs]: https://github.com/exasol/hadoop-etl-udfs [s3]: https://aws.amazon.com/s3/ [gcs]: https://cloud.google.com/storage/ -[azure]: https://azure.microsoft.com/en-us/services/storage/blobs/ -[parquet]: https://parquet.apache.org/ +[azure-blob]: https://azure.microsoft.com/en-us/services/storage/blobs/ +[azure-data-lake]: https://azure.microsoft.com/en-us/solutions/data-lake/ +[apache-kafka]: https://kafka.apache.org/ [avro]: https://avro.apache.org/ -[jars]: https://github.com/exasol/cloud-storage-etl-udfs/releases +[orc]: https://orc.apache.org/ +[parquet]: https://parquet.apache.org/ diff --git a/docs/deployment_guide.md b/docs/deployment_guide.md new file mode 100644 index 00000000..66382b7a --- /dev/null +++ b/docs/deployment_guide.md @@ -0,0 +1,156 @@ +# Deployment Guide + +The steps in this section describe how to deploy the user defined functions +(UDFs) jar file and configure the UDF scripts. + +## Table of contents + +* [Prerequisites](#prerequisites) +* [Download the JAR file](#download-the-jar-file) + * [Building from source](#building-from-source) +* [Create Exasol Bucket](#create-exasol-bucket) +* [Upload the JAR file to the bucket](#upload-the-jar-file-to-the-bucket) +* [Create ETL UDFs Scrtips](#create-etl-udfs-scripts) + +## Prerequisites + +* Running Exasol cluster with version 6.0 or later + +## Download the JAR file + +You can download the latest assembled (with all dependencies included) JAR file +from [Github releases][jars]. + +Additionally, you can build it from the source by following the [build from +source](#building-from-source) guide below. This will allow you to use latest +commits that may not be released yet. + +### Building from source + +Clone the repository, + +```bash +git clone https://github.com/exasol/cloud-storage-etl-udfs + +cd cloud-storage-etl-udfs/ +``` + +Create assembled jar file, + +```bash +./sbtx assembly +``` + +The packaged jar file should be located at +`target/scala-2.12/cloud-storage-etl-udfs-.jar`. + +## Create Exasol Bucket + +Next, you need to upload the jar file to a bucket in the Exasol bucket file +system (BucketFS). This will allow us to reference the jar in UDF scripts. + +> Please see the section "The synchronous cluster file system BucketFS" +> in the EXASolution User Manual for more details about BucketFS. + +For this guide we are going to use an example bucket named `bucket1`. + +## Upload the JAR file to the bucket + +Then, upload the jar file to the bucket `bucket1`. However, before uploading the +jar, please make sure that the BucketFS ports are open. In this example, we use +the port number `2580` for http. + +Upload the jar file using curl: + +```bash +curl -X PUT -T cloud-storage-etl-udfs-.jar \ + http://w:write-password@exasol.datanode.domain.com:2580/bucket1/cloud-storage-etl-udfs-.jar +``` + +> Please also check out Exasol [BucketFS Explorer][bucketfs-explorer] as an +> alternative option to upload jar file to buckets in BucketFS. + +## Create ETL UDFs Scripts + +Run the following SQL commands to configure UDF scripts. + +First, create a schema that will contain UDF scrtips. + +```sql +CREATE SCHEMA ETL; +``` + +The following SQL statements create scripts that allow importing data from cloud +storage system. + +The first script, `IMPORT_PATH` will be used as entry point when running the +import UDF. + +```sql +OPEN SCHEMA ETL; + +CREATE OR REPLACE JAVA SET SCRIPT IMPORT_PATH(...) EMITS (...) AS + %scriptclass com.exasol.cloudetl.scriptclasses.ImportPath; + %jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-.jar; +/ + +CREATE OR REPLACE JAVA SET SCRIPT IMPORT_FILES(...) EMITS (...) AS + %scriptclass com.exasol.cloudetl.scriptclasses.ImportFiles; + %jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-.jar; +/ + +CREATE OR REPLACE JAVA SCALAR SCRIPT IMPORT_METADATA(...) +EMITS (filename VARCHAR(200), partition_index VARCHAR(100)) AS + %scriptclass com.exasol.cloudetl.scriptclasses.ImportMetadata; + %jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-.jar; +/ +``` + +For exporting data from Exasol tables to cloud storage filesystems, run these +statements to create export UDF scrips. + +```sql +OPEN SCHEMA ETL; + +CREATE OR REPLACE JAVA SET SCRIPT EXPORT_PATH(...) EMITS (...) AS + %scriptclass com.exasol.cloudetl.scriptclasses.ExportPath; + %jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-.jar; +/ + +CREATE OR REPLACE JAVA SET SCRIPT EXPORT_TABLE(...) EMITS (ROWS_AFFECTED INT) AS + %scriptclass com.exasol.cloudetl.scriptclasses.ExportTable; + %jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-.jar; +/ +``` + +Similarly, run the following SQL statements in order to create Exasol UDF +scripts to import data from Kafka cluster. + +```sql +OPEN SCHEMA ETL; + +CREATE OR REPLACE JAVA SET SCRIPT KAFKA_PATH(...) EMITS (...) AS + %scriptclass com.exasol.cloudetl.scriptclasses.KafkaPath; + %jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-.jar; +/ + +CREATE OR REPLACE JAVA SET SCRIPT KAFKA_IMPORT(...) EMITS (...) AS + %scriptclass com.exasol.cloudetl.scriptclasses.KafkaImport; + %jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-.jar; +/ + +CREATE OR REPLACE JAVA SET SCRIPT KAFKA_METADATA( + params VARCHAR(2000), + kafka_partition DECIMAL(18, 0), + kafka_offset DECIMAL(36, 0) +) +EMITS (partition_index DECIMAL(18, 0), max_offset DECIMAL(36,0)) AS + %scriptclass com.exasol.cloudetl.scriptclasses.KafkaMetadata; + %jar /buckets/bfsdefault/bucket1/cloud-storage-etl-udfs-.jar; +``` + +Please do not forget to change the bucket name or the latest jar version +according to your deployment setup. + +[jars]: https://github.com/exasol/cloud-storage-etl-udfs/releases +[bucketfs-explorer]: https://github.com/exasol/bucketfs-explorer diff --git a/docs/developer_guide.md b/docs/developer_guide.md new file mode 100644 index 00000000..aaf5aea2 --- /dev/null +++ b/docs/developer_guide.md @@ -0,0 +1,166 @@ +# Developer Guide + +This guide provides development workflows that used to develop and maintain the +cloud-storage-etl-udfs. It is intended for those who wish to address the issues, +merge pull request, perform release or deep dive into the codebase of the +cloud-storage-etl-udfs. + +## Contributing + +Please read the [contributing guide](../CONTRIBUTING.md) first. It provides +instructions on how to fork the projects and get started working on it. + +## Prerequisites + +You need to have Java version 1.8 or above installed on your development +machine. + +Additionally, we assume you have some experience doing Scala development. If you +have any questions in general or about the development process, please feel free +to [get in touch](#getting-in-touch). + +## Build the project + +First clone a local copy of the repository: + +```bash +git clone https://github.com/exasol/cloud-storage-etl-udfs.git +``` + +Then run `./sbtx`, and run any of these commands: + +- `clean`: cleans previously compiled outputs; to start clean again. +- `compile`: compiles the source files. +- `test:compile`: compiles the unit test files. +- `it:compile`: compiles the integration test files. +- `test`: run all the unit tests. +- `it:test`: run all the integration tests. +- `doc`: generate the api documentation. + +You can also run several commands combined together: + +``` +;clean;test;it:test +``` + +Additionally, you can run `testOnly filename` or `it:testOnly filename` commands +to only run single file tests. + +### Running E2E build script + +Inside the `scripts/` folder, you will find the `ci.sh` bash file, that runs +end-to-end build process. This file is intented to be run in continous +integration (CI) environment. For the continous integration we use the [Travis +CI](https://travis-ci.org/). + +Please run this file to make sure that everything is working before commiting +code or submitting a pull request. + +```bash +./scripts/ci.sh +``` + +Additionally, ensure that the `ci.sh` scripts works with different versions of +the Scala programming language. You can check that with the following command: + +```bash +TRAVIS_SCALA_VERSION=2.11.12 ./scripts/ci.sh +``` + +## Checking the test coverage + +The `ci.sh` script also creates the code coverage reports. They are located in +the target path, `target/scala-/scoverage-report/`. + +You can open the `index.html` file, it should show the code coverage reports per +file. + +![alt text](images/code_coverage_example.png "Code Coverage Example") + +You can also generage the coverage reports using the `sbt` command line, by +running: + +```bash +;clean;coverage;test;it:test;coverageReport +``` + +## Checking the dependency updates + +It is important to keep the dependencies up to date. + +You can check out if any of dependencies or plugins have new versions, by +running the following commands. + +Check if any plugins have new versions: + +```bash +pluginUpdates +``` + +Check if any dependencies have new versions: + +```bash +dependencyUpdates +``` + +## Releasing + +Currently, the releasing is performed using the git tags and artifacts are +uploaded to the [Github releases][gh-releases]. + +### Pre release process + +Update the `CHANGES.md` file with the summary of all the changes since the last +release. Similarly, update the `AUTHORS.md` file if there were any new +contributors. + +Please make sure to update any other necessary files, for example, `README.md` +needs to be changed to update new versions. + +### Releasing steps + +Follow these steps in order to create a new release: + +- Run `./scripts/ci.sh` and make sure everything is working. +- Add a git tag, for example, `git tag -a v0.4.4 -m "Release version v0.4.4"`. +- Push tags to remote, `git push --tags`. + +Please make sure that the new version tag follows the [Semantic Versioning +2.0.0](https://semver.org/) + +The next Travis CI run will detect the new tag and create a new Github release +and publish the artifacts. + +### Post release process + +After the release process, the new [Github release][gh-releases] notes should be +added. It should be same as the pre-release update to the `CHANGES.md` file. + +Click on the "Edit release" button on the latest release version on the Github +releases page, and add the release notes. + +## Editor Setups + +We try to keep the codebase code editor agnostic. + +Any setups required for IntelliJ or Eclipse is out of scope. However, this can +change when we get contributors who use those code editors :) + +## Getting in touch + +Please feel free to report a bug, suggest an idea for a feature, or ask a +question about the code. + +You can create an issue using [Github issues][gh-issues] or follow a standard +[fork and pull][fork-and-pull] process to contribute a code via [Github pull +requests][gh-pulls]. + +## Conclusion + +This guide is expected to change and evolve with the changes to the project. +Any pull requests to keep this document updated are very much appreciated! + +[gh-issues]: https://github.com/exasol/cloud-storage-etl-udfs/issues +[gh-pulls]: https://github.com/exasol/cloud-storage-etl-udfs/pulls +[fork-and-pull]: https://help.github.com/articles/using-pull-requests/ +[gh-releases]: https://github.com/exasol/cloud-storage-etl-udfs/releases diff --git a/docs/images/code_coverage_example.png b/docs/images/code_coverage_example.png new file mode 100644 index 0000000000000000000000000000000000000000..36f7d65e666d7e3458882169b694f76c9fe13b95 GIT binary patch literal 163096 zcmdqJbySq`_bxh02@)a_k`jVQcY{a?BB*o@-QAtiE!`p@(%mK9AYDVp(A{}IoO8av zpXaRm?_GD@J8QYXncU`T!t{R)Ag*gznUyPqI~ zXM*39wu2vtw(lk7pMZzk6TLw2|Ce@R%64*=26m3xHu?}l3rll-23uVleSHgCBTKt| zgl0hqFye}Zy4iS6yHDorP4MJVrf8$k*4B8sOutfJqs}fPYos%LN2j0QfSs_E06ji8fa0G=JT!Y$GN{cPgGR%r z;SJMU#6Rd->~B9{M_uVNr))QNNPbm$Z6QwSK|QL!86WuHj}Ia`9bV&lS%0+QdXh*> z;s}@lI4f&=34u3FdNGQN#$##`|NZ8`?rDB>@v>ezjpEo}f!$1b{Yzt3z5kpAg^&&l z!5cnPMRlawf1jqHzzFqG$Nu*bg)A;y^v|IK13w^SzETqX_mSoOC6excJI=_TH{H1Z z9tPW3>&0sD|8^X9W;7eIe-9%{NbvvI>7=lA+dtM_m~HzJUtMP8Xi#Y%a<0d^u~uj% z`{or-af&dLeoYdBLvPF`N#)Rk+&jSbn?yP$x6q8REqShe7J?`~sjyE$9UL{3f)bC@;f zCdP!sF+pWUe|@55U}#*m^+|X5^zYSY?e)V|YfeQ4b1&B*ZslyL+9V<#!!kkY7C8ug zcqF{GWKL81Fg*_mofSv@_f{ibV1GLH*#rlt#urluyC=lyMk z@u3Vh%R*IzEzee^Je-&wmZeoYGq3(O6cbDC&yojAeJ+)q$e-w3M z`V#VIhas1KCRNx%^7ViSzq6ASJA;UThIzaKuhUM52$f#^(Q;v-5y_tW{28%KLen}` z-?`z+iiOU*?RSE6DZN7CRX;5IAaOf=IupNM*nE5&WWL&wjx2_avn=`E;bk!+U(4{x zb7}F7zwPlolbn=)j#J{LJw!az?OkYF^oP;}HmA#JAVUJ@#0KSwJhtmvm8zxFW{{JO zVFV^7rciN83kwUlnc?yJ0RQD`wB6>29|@lWUXInnW&3QWh2# zf-rGnOq8`X1!y~S@_=-@APE^6nRo=L`xm%X)m05OMYCeQ1Lo|YO-Y`f4C zZLqRwiHrjx}5_jXw=emw|Q9caxbhT}X4{;78Sh?T#jTm08nF-kW zaKVdLS7>O!O&H6oDM&$Oa&K>@hKWjxZiwyj9j?865gV6ktC0zT&R>isTy5y263g~z zb$&lh`Ow$d)t1^aH)A8$T!cS27@AG=VTfU|hp(u9!~Hz%gU(W?Mtiowk`=cRDQWA5 zut1*1q%Q4dziLy^MTHr?0h{|w<>FwAKxu-sz2G3%wqwbz760L;3gWfPWX)h-)t>G^ zT3eRhns%k~#*O8vj`rkU@~ z?v%iXPE0vkO;*`^c;^4EIBsz?<%vrXwoznMax@ueY(r5WtDW#`+tge7HCHh29NB;K z{&53;ZJn*D#ohvjmn-!4+s*QK!s|wxh25PnS)I=B7;y9JmMDbHN%Auv_uqZxQQuR$ zu0tkm*l9_GWU<80Hv1PXcGb6|S#2+E1{A}ov!m7I-3E1w=WM@S5;=U2QhMhVf#2q8 zuLDjBe<%Gv5c0v?1hMw1gZ?7oRTcW5K9is3-?);)*P@c3GBxR^Ke4P#&#jUWYuM@} z5RV-UN*J2FgfK#v)48fv18^?s=c6Ts6EX5@x3FT8SB{upVo2|h1r0+qGWlvI9sxXu z{E*ASB1yV{Ji}RboW*L{(nm zGoh{N=E)rpAjJ3Knp*J082jDi#>c1SYm=ExzCB8s&Pe6c{k~X&8>(lUIq78%yQ=eU z$DeL)ZxfldA4dq?ao^wWd*<)*IqWDmG&DG^MoBpBR!%|y%D#H_3KHGZ-s<;sy55oL z9wdhP^~KfI(QfrZou<&uK^yu@Rzx8oA(xX4VHX!(ETJ1_@XsoBD1%&azUgk9HJmCN zXBqxUbmjBc^5ev6ZJKKIjqvHur}aYcYjd|s2O`Kk>7Km2V%hG8$2djJDFXPpWJ953 zOGyaVZTk~T2BpkrNqJu&^y_sf(LxH=Dm?dzjR}fT+Wf9cWHP@DkB1$%Ws&N38DZ}P zrah>Fmmyp4%3IHzcErq>q*S%)$sHAFD7;0 zX*0bYj>avwlwc_$_aY-O`1-Cchq~ystf>iHp%12yo}fZafDl41ZDd-0jCmxQLz4) zINOn-k;%acz>zjQUD*+A6euG3^cY6_rhIABpw*1sO6{Ckhse2=WqsXp35$Sh6x#GP7l`+>hxZmq^ITfcwd^LVvu(QaSl zo|2X=_f*Y(|Fx>PCOH|?+@Gcks!)Q>sh_Bh7@HyhM8vY!VAosduztLG&l{|SP~_gX zZxb&@BoYRw=ab#I@S=k%QXU&HE_B~r=+PVCFc5L+>?~@EBBtiTJPkVe!>N9;SXfzd z?(j*y7w1ei*H$DDB#ky*f&QxC)9ecjxFYK;s2FqVr zH>Vl>owpq#mDHuSyP1E55{1t#t=?KT8$#V!dHgyqT%E1#;heCsVmfhOZ+qA9gNYl3 zqN=?MxE&;k6K&Z~XnsqziH|XgN%8ibjsYf!XLVP7a7*UrD+PJj;vMk{$r(99eueoM z;gfuUZ$D8lNuNd%Q2k*Lj>-*|yCMyth-fsx+5b zh@K>^X{$6lI$PT*-+-uJG--DKLg=EUD_(3ND()OL$<|*HC;l|3b$NyWGG}bZ z&7ct0?u|oBfd2RG+mxWmbIR_?WJkOr?b9NWB6Hb=YSUt~p*fN3v0O5tc}7+$FUexJ z{yQAnlkI1H!&!D{AA1S7YPjY}6)AjfYD+Eg4T`7Q6NCjc_DS$ZuJG8E-T@|@lin)Y;K@6izrW!Z(^6nO%m=x{?gfx{!$z3 zTzzsuA(4;-Px^dfAkQ2Q-~F;OjZ~5#aBx^~m6FHJ@mkpF^SesJW=6&nvz@Sy6bv-! z_e6Xt&x9077=v(U&+C0)c-Ub^*uxw0PdR*HBoa7*Xc#t)5$tFO`i|-pbdIF* z|J{KJNmjmcJk2me>otSHDzOEU(l14po`$R_H*iJQUyt2TF8R4t2%=`T{26gqXoEO} zoaw@Q9-Cp(a5-3r>iysYEIsg8zG)`kPiQ-Q{irz>)h_sG?Y1(A0G-)ZM*>nOetk5C z!P2fD%W8fqS;v*%*jV)GcFKu9;=WLNvDd)Et&XQ-yqa|2_7b!LWJR*;or4(-W$dK& zZ9OzNe9ZR4)@L;MN1kmu1Te-x2_Yd-?b_R)Lr@Qm@ZD+W!NsGmkNYwuBEBaid}#K# z&12Xf%fZ*%pIwzKBdkC^SZDKtk<|s;=^W#7XGWVDOQkTH)?hiW-e0U*9Ass$#OI8X;<6`cb(N~KG zEGq^qUsN?)W&{E@C!26B@Nni-#pJG6M|HRYo#kjk8wHq->C`wfB|kuLV;z2Z)CCSQ{Jc2- zruHWP_hRNgC$nisCt+JlT7Q4bZ358{jN>_zBiC74-l8oxNkXXj9to${UkLQ8?UB=Q zax6ej<{CA=EBnA~XuyTH404}?59__DK#AIJvTp9GSC+o`(XeC^Lnj&W3 zk?iM)L6;8a7sW3;KuA$DLG3m!{4NngF za4M?Jh&w?A*oOTY*!yEvba$=8lKA1nKwbc8$p0c0-TNvaqvr?h=&2EOUBwr-tqsNl z2QxqtC$6ECF^`v;tW``gGij^(l?WZa1qO$@ED*4|LHh3 z`n%aDmZXHlw{k_5H&Qp;H4S5bSOo=1y$lBOl*n#Z$nIY;F-71*-J*JXrNqODHwk&2 zuTC~aB_xg)eX(o+bFMp6Y4#9+AfRBj_w|JY2cvzJlQXSwpDEMto2oR$f1oUohhDMf z_H6bgn*sWe;I&yL+EU%`V$-i=^wc<>HC1F}^qO6+3q@QiXAa?Q@dXl+|AOeAP?y+6u=co^?7nPq}stZqqW0AeR-7QfF%3E)vVogv(_j zNxM1`lUi7;;>LegF@_iK4=jCtj|RX4)~a*XL9?=Fuyh$zy$F8vIKm@SVz*5k@2*m( zNM!cvn}lv>suA&$Nc=-0d0l-xBk$@AYw~POHef|#JG%U6 z%5IT+l$7-24_SpN>mmEwn(GcM?x-ZmzD|W$6AL^x%4@q$J$5Ux@=nlY;>0IOMn^)L z7HkPZdb$NB&uj;D`CUF^XuZ(|>UY-sQLEmfao0_9Z^&Xu4zRN};lckfPc@Y-cnA_w}=SZ<)D*2pWD&+#S8Jg|lCn zCD*>TQMH59-J9;h{M7>WD$?K;rcC5?+gtB%FTR%9*5=7=+k)<>>^S70qE6!g1}~WI zP2o_}C@!>EPq+8jCKDn;>d?P%agKzlx8m&g>&rI}UbxiCeO6b6bIywUm7fGO?(jQ& zmtpw=I_;9v1JqtRI?tlfXa7s+j9LgX5#F0!8N5YVFBPC9m_6?Q`+d^!iS=!Rjm^0# z7ITFLbHvRX2*$DPN@1z_Q2kJZ-imu~n`eJF82%$9*lnC@Xfnru5aS zg+O*n-5kxCRF02{k=K1GudoFa%<-Un81!qzFpBa?pmlp#i!nljA=l-CJOydphV_Ft5@m#rc}A zxszq6h=7w2od5&b1})gxaG216OtzrZUB&tJX3k8I`A9czT&3kimTF|O(-ypXy>PYA zna*=8q7JDxm**0F!)pDOacZ(&^}>dglE@21GhIPIOZm3zBM4DmYMQF-IF+IgAV>?77AwBHp93My%sdRr9a0s$ae6mK`u z{L%@FI2o!Ta9O1Nq(beih0M`?L)agP-){#-%XY1g4!)?_tgsC53Y6D zt`dfxdJ4(#O_nd&@s`f+Dl^~#-g;Q3N?TY{1F3S|HbCE>i;8bn_ZeNH`qLHNw5tnl z=bs^K!vBc#M|?U>2h`0LsTA(Y&aaZsC#k=F)ux%64GTb)z}Zteuccf&&m~8U!}ugp zN9`@B2^(o90?@Hu$+e^t^UjiManlGRor(tF zpK6=>h&eGYy(t z6vDZ`K7B};hY|bqZ@xKt<4k9yoN{PxY7m8(m;fj?&Zjf{XLsfD z_Hx<>mR!%dyj~B+}dJ->2hD?oZ3X#033 z@S%|ErBg4?eE&H2cLh%Tck98(I>w*xA3qT7BQzq#>W|kOJl|tvw!5wBI>hVaUZ+wP zR406z+JzS(hoWJuF80J6BiDvKcBd^aiGr&8q$?>fr^RU;l%I^cy>VYVW4+tEQ+|E;ivdkS&t?6RdX-zr(oK5VZ9H}@NCB~#r%;E+A_F-=3rXb zTUciqFNl9zW`2EXvWp3NZt<1HW2K+zMKjQ9IJ365M-ln-wl9!bXKrEv(`V6#VK^xd zxi@jv`=+$&@E61pdOK=M(#jb!{shFsah%n2FHa-jutIec&vlRdEMv=YIxs_XsI-VT zCGXur#XW&%S=hDS28KYvblK$+XOhM@(Pc`H|Bzh zC(Yq)ueUFcuFd6V%$(}ffbJW&SD1{Ay*ktqoqMx@0{voK9i0#*5)vq`j_q!6Flc;6z?zH#_^|qcd)(c(l*L7(H|e*0RXQ!5sSRi2tQNI-bfgxF;W>I%%;bKq0a!JQcMxphJF_X* zP@mYVliOA!f-nS2jDZ1=&TgP=BKKkdb)ZtrWejgH=h-j zOz-Xpe8COJ+8>Wf2>hJn7r<=U#@LCmC*~sai0F*0WA5vDL|@(6s0j)88;#NP z=U&rBtphfP{h|N)m6u7$f9DQwOd5mTZX7?5o%=VlkAl7zwGa#bn4>J5MRt-!|iyqP) zZC;$Ao3%Yxnas;-Nbww0|5lw7E37dS8L4ScYDk8V_cmW?pa0!GCId_Ipl8_K{%!H$ z`Pm{+qEvM?fCS}G?+X$f^jqN5miE@y(b$;mitwKkF7{Q<%^vq0g>e(hsB#*MDtcgp z%T2V>5Snzj6w5BUwm)l%=Pf>n(Ab!+`Kf}_`uNt#^H6o~w(%8k7W_{y0P&b_rsU=h zPgFOKqx!x{=7>QtXO_F7F!rIb%6M9F@q~Gf!iqS-C-J7)vUu1}9shIQbA0jt z6xa&C$#YUJggKnX-dHh*XB7~KA|n6|;N^V1Zsk_aEV5nKP@nJG?0Ixjv}c}8A?;@c zj?>BE$>YN^jGYkKPA_j#^Ca8zzk=ib3FgyT*kHY1yoi5}w;{2j7i;A;Cml+LtG$3*SM?#KgqbS2=5~zWX%* z?Km8h{z-&13NU>8jJUzixxuV~AofE$vHp=!E z!Xn~}x|Y60HS*a~@ifjvmYicWCk-i?FQxjKzIn?)Gc}YPK0v9V5Rb1@nQ+-$+LlYnt1Sy5dViV?6GRPDK_T_tcHPJ1`u4Vn$e0Azq2)~0 z6BPmUnawr~Le~V4>*Z*G=2}{CPW~5dX}{q5Rfe_6_H| z=iqYqZyI1gYrf4R5dwW{a}(F(=~c+M&_Tx*S)KrtcG}sk-WiLtJ`7-18Yx%9jQZ?< z=psdYt5!8%N)Ey!U{GN-93M;M=rLfoI6xg3-IsoPybj;qS9qRQhq>~ zprVQ-!%Wrc8g_!{P`t0b-J;uON7?U~s=4Qe7R;c%3+c?@_qevVPfM5ouY?9?k`DYX zQOE_J0m@g7zpwer;#+k%4`Fj$cLiPl>FWT~iVD%4Z*u2LqWnLMFH3)G68~$l{>LvE zp|$SUXL;SYQ7-uS_>Q&pul{-cC_ft%;(s6ipRm`Qb5S!iFWWc@XDRN9BgD$U*Y^DX zv)=`Ne(?ReX}4^_UZY9ZEf9fJU!K4!v}d@G!+2NC8+1OAt!`Dn+F(O^wfuYgba#Ib zu4Q7~crL*>aCdv1$Zbsl6rR-yHB0@&<&Glzj*brg+mlgs+X0SH0u~>p=9{C2-4t$X zAJ_fH`26CgBo^Jy60pF$m9`fL+Iw}|jKEznkZjqssj8`}+FN72^5*T^JUE+R^=HSY zuQ9urZ0P@4?kH%;*BoE1{Ekyq;XTU_D%rkh^^eQ^w=6xci5@IGbczLwiH2!Qm+*8N za`H!BlbWs=$pf7?m&eXPQ%+>jMRq;kD;)M8FjxtqN&De>e@CxT7jwAM#irl;Lh9p3 zrm4Zumuxl1gM5hsuAHWmMfN1F69p=v#5~U}E%%|TyrRx3xputW)w2qT{~D&6U1Mcv zE*spDY9>cT#cc(mZ1*ZCz3A0NMP$c zU+vO%K_Tpsvd61{p(=yVpA9l4y<%Nm0!A|=6^tsKEIFaO-LIO*WR9Ov+aTTTYrLgAC`kZe7G5TWvLeX; zhp4)m-}0VBCZUui7NhH^o3?ZbtczKMxcVq{Bz_##vsDfI^o+N}O)HJJH|OEXf^*;f zbCQF|mY+U3w_RCNQ&k^1y%u;&OH0nihU44f)uQ#sh+T&>nI%zr%8XO7K)D5l%*`u4 z9=}kn+MqoUyFXnh{cLwu$<%P6$^8W`E{j1w0Xi}Qh!=(hw_}Wa#X_BpzZsQPx|OQ5 zjEuhu3eb}W{5y2W1q4#Vh`5gCUH5NwCR8l0h<#RZ;jU%>A?o-t=Ie!0~`l>a@UULZOEnIqS_A3rXw zePOgcw0<@cYO<1uM00S?%$8;TDmp;)uwALE<}Ig}irR<~{!+(cl{XbU_PDcGRI{NL zsj!V7c!@gOprqRG@*xxm1vicg&qw#n#PqykFY8&IZS9}$hno&0Q98Rd@PxtrxhSc} ze8pVL5?PtT9XqRvPId0!^@vG`%83G{Bs?9l(rK=t8$Y%(PO__Z5${S55CxJhkM_T? z5PzUu-$b$&wfk*yA7cK}j5lp4gsW?DTYLQ!H>jbJI4jfFnTVuJvy$9T0($UnPK1fSv{ru_k>r--FRYZ8W zL0=pL$Bg8M4?m1Ye-WVjo6M35d8CYujj7H)_(GT(4j$XW2iE|;y{4e>@w{0flTKns zEx+_x*!-=e7#*byb=0xG7mN584USB|Z2#?5!$CtmE4A9i6$YE#F6_r^e#tr|2^a-} zF?|{WZBNds7B>x6@A3)DSMz^D|#Jj*x7E(O2be5^H_HeBHfwBvT*JJ8M1 za)S++g)wpObM(9^SbPT>ovUvv7A((~EY~iRffZy~#k~HbG0`FjSK3S5(0$zS4!s{B zIvaIw)VJeFy0!ab!;Dt9{R29>1>9Rz?BB$rsmtDzV~*Y74B2z=aL_l z-W3QWNkl{FT)&4n*;&PyF1#vcgUiqINZAu{S?JAHnLpV59t@Pej}VYcoDOv?>UTNK zIEgru2XI@{=t}(9GIpnX`}z*g&N}Dj8V7eF4VUZ5{AW|T{V6=r4!bkfV0+gZp6WPK z?`Y}cq@~=;Lm13N571=ql=jzM5i|GlwtLl@k$vJ5kVL#KfS=e(T{aDN!fdMK# zsH)-PMtTAaX4Cg-N|CC{iVE7UGis*aH)dNS#sh2##%9tvmoy8w6WY!tj+}0*@V;Hh z`}OJ03qjs}sZB+z_ie_q^dDH#Q*$G~zd1uprD#MwtBSnK>|Lz5LA$DAmQ&p19^xqd zZ0z{;vm$QI`*G%8O^b2gg_#*3Zz~A1S@9Mz&=>P|y|QGx%#Mvw;8mY|RbTU52#d{M zZgOL|CCV=7=uzJLX>hdG+k$q($$~}xctDpvYuhp(r{!&@#HbC}t1C#6BiRZHMMsMhaDVP!Cu^>7u+ZnQ8lX>OiD|0waX9#$n0OwFXra^^%S$V*E4`ED-Ij8elwtb|O> zKTSj`$oKfKTtY-I)Qm~iJLO82t&RR$oG+#<)ZYRK`e8E zVg`(UEa6E3R_W*z9iq8R_x#i0b*r*HWkIBDPM=1G3d@~=&@f~9sg$hM*yEO~7wG<5 zK;WGv7sR7CKnOY0`=YwJm`{@UB5np^M7kqfu7xECi+rCw*K-%6GeOeFmp=y)oK6up z&<5gYbYk*|lZ!Ahuw8)-P`?4;uNKZ>!SI@buV>Wwj}(@ws>i>L>ttoMw)77uOjeI* zR?3xYIo|L~yOlCT3l}2rqywv;x5ZptR6;^Gpqjp;n|i=|T%cU)skBtOeO` zIK~77n3&HY6zM4*v)#sL2rC;KxSbe~-P!7{xw%yE`xh0Jo?u)$ZjT%8+pE)hExsHD zd3k2tPIPLSlqc^*MG=Ad7H}3;R@Ns^pI(a3vHAqH}$dC`r=}B|IzsV87$g?HV3V;FVKk+6#DT zt!pP(3>qb11^MGeorN!d0FWZBp^7%mLR{P)Q+r@Z9V}+AMq}%As(kxJ1SY6fKh10K z=aR0hSaU+}7;c?clRn(lzl}~{nas6DGMxRyI!qprqGft%9OY+!-1ju&Y5qkVz*N9tPpCQh`=0-ccvBJ?r2;^9uCS zRkF5Z;OBYZgu$Kj{LI(PmM~JaUE;XiA#(DZl>5-5^&-ushPoRn$A)S$CN3ObRg`BN z`b&B;FCV2T91~)hkaX{mjciQc++^(d4SpFknyH|@zZ~*(0%+#6U0D4D6?LiD^Og=+ z$PpJyyGU1Jl{A;))GS(bO06a}9Kk#R76g-&zdMFjP8sZe14nv1)Fbg>n;bYy6U4&^ z4{wf#R`&NjTO4GrmqVBcSoNNGdn=INCa~x_0qcuOiPocsMkXU8qj@p*pUww6%9bT2 zE{-Qmyv?ur1$OO|WB3paXVn!x8n6AZbpD*F`;KfX7XD zeh@t^#r5K$gV$6e9XjC%!iO<}8JTOtJ;z4U({&lltQX@`7E62Rq?b3J8_%o{)A~P} z*j{>@-Bq(@XkL$e-konuwqEHRuC|1BMvx5^Y1FH4mz9?%@;kF4Afsz_N0K*ynUtyB zGu7GII(xe6c^A`4eb=*{47F;D@w;SBb3#Y}CW%ge0t?KhkD=kPlc>lY@U_~NaK1Nh z^DYb6?Z7C`gJ2L}PZX~=-{6c+!W#p~27tlPzL9`QsAsC@-En&S+7tyvGZ*}-pVw*; z(&mq`0;XrQ0WpcsH3Y?t+@%>08@slIL9L2euRGGk)N-!Q9_F+Xb~=$3*j*g!16zn3 zLF2=9=vv$B6r+7pxzz4lTYp&I*VD6<5kUIT;T5P>27>{I!-F=g=(srKKc>Zv;-5c1 z18y~a(DxT;)W=;O!_&(d0@_+qcx;_ch9#`I^A`36T+jZ3wQ0INQ=9Z$>54!Bo)I#cJNEm_$=+NtHl_H%VHa8b1THS_A^0d( zz3yiZGd?7au$DZ9{Jh;}<5B9phGUtl^ZlU`?Y0CSTZ78ehj}8fW|^|-g3<}h-eARV zDd4Z|xy)xO2%$f>CJKrC{QSnN_$Cb%0q0YxFv5-yyxL%z*2lg~b-BBQzZ7)iZge@- znJUq403JQ=RKq(cJqt?|Sg+xHB_?3{d8lKIx|U24xO9rlfMUCxZgu?45OD^RB99uO zA>Ec;x)B13D+`62o0~l!dhZc$uVsgkL|FaC74ar?m{w_qw(owwSn%Eo zKg<*eVFc?e6`_ag$1Iw29QOwjfWd`HZcWXguxiEEB?1``z|NOa-`B46TD>*Hyt_E` zxavY?&Vb?^eKeQ8Fk*_nl3+av)`}zRHW-{>2QxJ+FEP|dIRjBp#2=N|LGjqQvVqt_ z$@(`{=qw^jHwz0A;$V2RHX#r6oc3=nM>)b@+P^tXxO^%Le(UV+%wI2q2Wsu|x`j4u zUZ9}U0d9INsz`f21RMG;QCC~lxb52r<9hX7ljV+TE$;B>I}Y){n+$~nd`(51u-i`% zu!AT#9n)Iot|C80a$>YMdRWcRR1D?XlqNMPY-kvgfby4-ftI-tJ(MUhTZH47HFmP4^tA zGMj#HWyK7_3h4*5*~4yYEJ(jEj>Y4~>G|{LMzd86$i9yTRJ*xTU$z?ww!KHLaT!R zrkflrw#3&zji=(`iU)5{tuh0tOhF+8=J~MYeo1ElVnxhr7f6%l$ zG|0FAjD%zWjNv)$ly+TQZ5IPEDgJtFjh=+pZV}9W6jkkXrLQzk4AEm>?h>#YJp&PEaQ*&r`rhKU(G%~6l-+zLCi`PNE0C%HmV!|fOW(J9pp=n< z2H2wEP#hV9u>B)bqurqSyvMaUuiXa3YgoDbZfwcw!D$O-uTIO)Dl_sbDXM>rb>6$% ze_J9EQ8|D!N^g<%EWpV!OZo>Z#%ZS0zqZe=%&?e#l>q$kUeyj(xgS@}6Pq&8BhE`@sJ6Zp2T?!gAoJz)NCjhF>+X!Ti4E%kzO! z`U_k9B4eEP?Y@;q52F)U6Eug_L^>B&vZ-K%!ixq@{HvtkjNCk3%?cfKWVU1*{o9$d z%!a0K)mD!4&24kd`Mh)P(;KhD^UH3?gzk8K1@u=$Q+Rq`Jmo?ow0I$*Y9U8A+Up+| zml(SeZyZO7pG~Ip=64NC_2%RkOqg0EdABB2Ilhq5s}H2haf5c32ddV_*&2k zdhXs>jBV}ByS5udJRN!W>8;g8Z)!z6S+L=~pS;1Dv%h|^fJ9SM0n~ybC<3Uz)|y7db>=SA~We4xh#$0X(`$8+j02D5?suPQg-of z!T1*K4Bbf4+h2h=ed875I}ZhyXx4z9MgLW4Ushz&2i}TKQ0>rH1wZp9Pdn0$d_0 zI=bEiv#7KGIL*b#_;A*L%k)3`W0FXjnB=L<@bK_FIOxD>K7a9|y*&u0XK;{nmKt=* zfB$|4TPrlzJ~AnfqysI{jD)zZ=u!U%0b zPw*cf8XBVE<|e3A{RB?v;Ph0%WsVe}jbfqdmvVzo^b8CSwbI(!It7sN;go_JrdTOI zfN#^XQQvzD$}VVn3jjXRhV^`VCR^MYMkH5jqfbmi0yN6@iHW$U_>2&%yBlXl&4%uS zr8a9Ts~=#41Pqv!IUlb}q-+Pk1IRpnfXxU1v8b-Du5T6cL?cKAdK>P6H~wLf7*s2M z7!IWYy(_73p-lfNsHv1-xEbi%$zjW1W^Q3d>}4*eChtW=P@g=}0_t~fJ=HsLabz&( zLngvP~oi6y(2hcAmVb({C_>|0v{rq|BkkEZU81_U(MI{GLe=w4qK%xuAkr0uQN2<)# ze*XL!vz9%swzjs$0bU|b`R{!Gk)dd=9n_b@;zre z3L#-4*XXjGWOK^2$1PwObK+GUxkh5*Ok@GLf=k=7zUl#fDVIPg1DN z|66d#X*Mbr`mU%6;!4L*U4E);G-kx{_GdVaHK$;5(U5DIo|QYqAw?#m&z5xTtV$$b z5qqW^M}p`nriM)YK8IxQ7`822$Wq~3ZS#y@4duAd^P*1}SbXKXcTzJD{AioNk9~%p zkAszs7?et-S(V&VR- z3#2uV^)jLs+K*6Dfh55AN_D$11#b>SK&D85SkppFDNw5p2c@sfbdu3vAW31F1eA6Rt_8=h*7isVT@ktD%3uN@Rv$q!wpi|TRookVL?fXiT@t95t*}9Yz;=y#G zi1c*mCecDbKmZDudDjIZmD|;3j&w5TCu3M_e0&!G#7ofbOal)-ol?;%*dUQRW7zc| z1*-w>NCK-q+VJpjZ=?+p(l7sI&=UPCFR5DKFDnf@GQ7hlw#i-n7dZkBbh&GkwY->4|8xTewG6sOTM*-MM4CB(NMVnfS{cg`Piyo{pW+ zx}r8(F~?9q+4@b|;E?m8mbsK%ref*lDnnJ8ROkE5pZ+-{{*NieGqFDw1{b@I8%(z( zhpM-&skOT$g; ziV2;JFLTpF!)PKI#?;eU=p>_6>Py>&O0lP^VN$gY`U>+sZZ$%(5_Dh7PUUO5tR|)A zrFhQidTQD8mb;7NzrLKvdtMc{#p<3T>nj!emPNZ2QSf?)P4MpGVe?A_^qM85rQ~37 z0jIj_C;nlosgd9 zqN57|!BysVY113a#zFd$--+3>;Sd>IV`-U2wp46zWyOp42_O~10D|Es_vyBNL)9}d zC^H-)g_L6xg5uo{0IUOa`hShs8-NR7tUv_^;_2!6OD47VA+yzMS%K~*kw5-DIr;iP zMk=NUB>NL|ba}yQi@7VZM7FK;x8-1?@uvEi;+qreH-GzF1f= zIyg801S!>SLk1kpFEv%86k9z_UM%x3Hy_{q8apyN!H=6}%|@4pwI9uq9sMDU0KUe; zq%tXAsrYM-3`t5#iqYR6-evB_=f@xkLDOF>Jup%X6?&j|9U=I+3i){8-TnR20A^*t z3|%gO^wOIi+lFV4e*gmn5L;dl5fMGmx7D8LQm}$RGm~`(2@I&!L!Z(yGBQ%&I9uF& zC#9%}XW4j)M;L|-00%ILY2cy(>`q@wtx%E8YVlo#Dg%(V4Z)5Y)$7+^N_9G(Zx@2| zo2s>?0YE9YmF;r2(*rQSeJl=oosd zBIQFZy(;~3%dHI3as$kxRECv|RmzF^2B}7iis4;10xAw+Wm9qQSTO=Iimz508yOp`IOrGbRA@i-}hi(ERzr;cr+$r!WMUj1X zI$GW`mT_z5H4bKOOs=7qk0>y8wqor2IqaRTtyiOvlia;ka3t?bn_;x~Ag|p)xDpO> zJ*cb>;w>vJEhsB6gePdbqT-#`s?k3L{uq~}8+AtDQ7_H)c!<6p zUzi@Ajrrb=2X;4FCLG+1Qm(ZR_L{o3Hs+1=IlH;p9bcPi(b=4;z&t-6kzX=5XNmME ztfLc^Byi@;3ls|LA_{(!K#aYbY23B;(a`s34lN6_AVVXF4T1StLldz=7h{g_UhR zv_<*yT3e0p{+8IYX|-zbry>44c_*G9v&Xr$lv8UV)x!Q zOahNT*8NNyBvFE1)@xEyyOH_SK~YFn&7OIFHaq)}+2Y)s@yv)kAV-85r{&v5jNtPgK*Z}n*z z3oo+U8rruB0r8ODA~LzcZsD&Q5M0!cAvUhlR#H9}Q%rh3OZ+VVo_OJOz?Bn+bKZR$ zD%^5WPG48C$)|Wx{YG`YS14m|_K~KjI~}y+g42JQZkqT8=hBEXXES&w_a88_=^x3F z2+n(5nX~q^VMb|pZX$&N&qLRWbQ4R#W^ zU%w44xutWGH~;Ytdx4ViCic3|cacvn#Y(AZhkr0s=Hfh@U)=KCV5_6>SG@X7_<)e= zluU-xl@r-Dk#9Wp3lJZ)8F>X)NxE`qgTf z`BfWLcYreUA#c2#$ADP6$;DlPpKJvV_!S3O-ihw9qmjaIb>&(<73KJtk13>5QvZ?Q zrAvF4RqWhcDV2W*SfH)N8hLuhUz;gauXpdnfe^Xv7B~>~QK6y0r6zR`R`e&i#N+JR z)h7i7L!LihZ@)Alg5|Zz-|6a7pP#BIzmuyg)!5jW?QFY6@4$d5Ag#^J(2qegCWVxj zD0wNc05&}MnepVyy`$&P-wr68xY0K{dh_1Bqxy|8x9vHNI%1S|uvT5oxnH&$3(6@v znlV|cis4QxKKW{Bo&SahUS3{RYqK4=#woarI&ucAv5k{yn}z0$O-=7izV`I|5F;s; z&Hmk{?(hZ-IeBKH3_%%t_XXchs@pDr_P!|ZShcI)Nj-}>=;j2M%o^kV$^+OtmV@Db z2?+@WO-+74>xnkA4>T9iwYHO*x)@3Vp)CM>i=7_q^9Xiy+z=cbT<+oJ)v#_{RZcEv zSJyw=v#4G3O?*7d{QNv{Im0lEYTm2(`0>#4j;rByI#EaaYN{W##z2p*0IP{D!cVelHe2TwvyNj}F=p`qcJ%W$k(Gfkzz=hM&cTzooTTX_kwU zPdMYdy~g@h;=2nhHLF3gG=~SSsBdSHe$=fXdM?=AH-}Ei=RjLr?Zd{~c`vhyUX0pJ z7$=0uKhoaF6<^?Y+wV8e;(nXzNVZsQ=JJ-evteZoG;F%dLUHY!sk93jcP`YMZu|Ro z_Rx47k5~vzMMHB*&eyoG6hkI)Q_6UO!;&7{`JVosHB%0M6E^tom3pR6Z)Fx__iEfB zQ+R5~qQCS=ZOYnh4BV5Alk8@TWBawcljo9bm^t*Q`+{PlRO_BJ+4J3vk^SrIRkmdI zOX5%D11lSQo}DFsLe%9+?1}>75O=>>=w5Gu?p7b%$QfE=#OgvVB)t zdVzJR#DCLL)w~g?obDbELBa$RngexA_*W!LHvL8>DJiMV+qY|hS5DFU;exVr&CWf0 z3bDW|%-)gWGB~&zY-*9BQLYdQV(-Vso_!U5dk!Cdg?)MNpszmovz&#YXar=c9XmW6 zW_nkTBPrP1=e5+dQy+1-i)~N963}bmJ=WJKykM8jv|r|-0Flx&H_?O4&cwu2tCy-d zwj5EX0~YNl6VoeXAJlop;86@XfA)7`U0y~N7r>_JQeQFiyDqd`Oa@zgxH0wwC#P>t z4j(F`mmwi^5)u*_Sy}Z!ikd5dMMc+U#yjJ~Mt}XfgsTjI97j)2kI`n`-QCT3cG>c@ zp!T3-A7f*iq~ktwY-L!h`d!(Ft1sqYmja+hL{c4G<1o^E9)!*H>(`O*+<-IfSC=h< z9agL;^qS&sft|x^b$a}4}pX2sHo}N!~b91GHX|^dVpTva+@mc_O zybuokQeVGYQBhIzxqj%q$(y}-k^e=C!G(sMirxJMBS={cnnv$J{q zSX71M9TMERPyJA&j_pM43uzfeOEUMn%jLYsR-!4XsH%o4Ca0!+GLu=9G79qS7O&s9 zaqQG7M@K3ea!WPWJ8#)@+PJrLS6`%#`xw9vf6|oIUg-@R9EDRA?k@OV**rmEzcR0_ z8C-H)XzHqu7HErdRsjpe4C7-R9UWvO6>I9xkyG}!=x698cz5@VF~r`?FLeZ%qjr#U(IiHV5~HO1FJ(p2hK zN&WQclXwu1590s(A9`CTDZ69SJEWYv*4){d3ONYTc;Jamww9LG^Tmp=VzTfG1lr9-gqWvLdsGULE0;IpA_yKp+_9si%*RBR~&8T}DO*D+<|y zzfxQlV@e3iLKv=vjEG|Jhe2LEIGfs7MK0LKV>vi?cI*0tgcxyh$nU~;&08Gfm3VjY z3!D^%K8&*7T~R9b^P0(OCg}z(UqNzYSWTTqIwlhqq5Uw;DpP$&wy?qh!J5K(zh(4G$^w)K#T z+4G$iVUdySxa{h8+*h@E!Y!~?z}&pJg$o&l8Y1v;2pTqH#o=(Pj9=1d4ASHq=l zqiFB28aSdE{1XcsSpyds0I#H%zyHgipd$duTHQs?K-yrdizoXkZrr?C4A%Mm`=dlA zN*<)`*nVn=$L+>HJ)*`ggX@s*0j_;>BG)8hUG*uLI|%S%gg znI@-jeJrUR3>YDSM)y5?_B_Ltf=xIqU@Nz!T-54JkwXJEBhbBf3mYhJJp5wkJ!kyk zKLA#_hA*9K{Yb@O`H_lcm!nO@1;`O^-n=0i9i4C^UFNG-TPdmBc9&9LS>j!hI4ZmO z*z=uN{_|hmx_9@MF6*yrrTQ;FCNG$4y8ho^jUUsplehZMzyGg4y)1dP-o)|W-u9m# zcb`A;MCpHfTg%G-3*MGwkEkD?MS^zxZ`Zfep_b?(YcsFBbohxZN2ind-@X{B@hDO} zF4A3j8Tk(qtA6@lUs0sNc;dvm*WCZZElK>p_LlsAJiwo4#IIzHsou=5-Fzcpb?p4V zFCOxgEn9CKofT-Llhj(us#skR&^&LW%|tcR!SpCQBgIxYKI^^h*z(n_fHh+6 zGo7?&WL4#c(vr=8-WzxPE0w;dxVWf@_yb8P0EV_DMb|3{h9!4Yx3!>pc`U{%(TGwz zICg}Uh5iUiMp$BQLTzWs{kt1-_vQGeb;e^qxj_|O3K6iI!}uicf9BOT#nZ{l%dgf4 ztlFs9P2WwPqB*Urt4lg@po(MIl1pp|xIIJ&tHeHdN5Rsr0YpW7ro+s%omafS)zlEm zNpQ$u^Rc2kqY#*g#DiFY*)CI2$tP$Zdxs+P{2^L9Hs!L$;Anfc-|N?}9hh=UeE76C z9}_+Cd8e6tlhkA+bJHNw_w?y17I1CwqBq>T)2eN znPgb7!@odBJ$H2EH|D|oU4A4{Efgy;N||rKA0`^Kuu**d`qdJ45V<^CZn%wZ&&a14 z8~}G!+|jYpnZH?+1LgxT`m#gja4Ee9j@ofdWR=;|(^Dr^!G6Av23LI7!0yr|;>OG$ zuov@Xvf9kOx4X<32^TQkW%=)LlVpn)9td&W0DROT!I+OUPh2X=u%2c>z6*mSN8lp* znxy1ucI~>1*XlOL93ZZ8MA@)NF*Mj2A`X^Ybbh|e2u-qlJb!g1D*4p&oyCZ5{zR<& zC`3i#min;K&x|j`aRin1cWWvmcz2JZejYMrW*jKhKOh6HWqI=biaa*&$OKA7PO=6Hj>LSqlaY7q?)HB z&xxFb?+BsfRzL&EY@j<2nz)5kMt9P&m<2VAVi(xU`39Du4WD9_5A*s z$qp7G3?eNO!LJYjM-d6d){T)5w5AG!FFpwC@9DbtrMACMcckZf-iOx->n27rUrSv6 z;OXRZWPe8zp9H5u+B?y%riufQ#o|s5e31F_Ql_uPjLos4%&J*lZ!YR8HN!ibwipv6>oJ! z(o8e+Zgw-B?+gk}CR*2`DJcvD?E6x} zAX9wm*&5foN1P$XY5m=HuA@9WB9a-(VmWU>++BRx)n(P=ywzbJ8PA3#dYmX!^ey>z zrEu<%ABv}XqX}KD>A2h7+%Y_FuJw+mvmX2~mHa?!c+7&i(4)QkcdEF=H!ZKXq1wjD zlh2R5)Lt#-crdUZMd9R-Yw&vh9;xoc5Mc=05WT6OEsKU_ zt|p|VeZ!rNcq~*a^NxMD{X!DWv^{|QqplqI{$*K`}Xa_ zA~Hcc#XcsvKeysb1jDhEJ$R@*!cnnGAoh8F*DX)fet=Y*c$=6&RtkWHPd0)vke}XZ>0Y!H8r(<^QM0~^nf9X z;D?4}1@-+K;tafnlY%xMKT2aQ4`7PrQFeAOAdX3Ndn7;X5k{R(`=}9-Sk2JzCQJiQ zo;-obpWD1J+vyN;@PZzkgIhtcm7wY%P!njxYJH~N903lIA@-!NkQ7)KE=)gRNOj!V zo!_0>q@wYw;7PeQV|4d)^6VWcc~5WeB3xwwF8_CP;tA9KN;TLFpxkeaP5aKEGx1vb zlawR$g?h*um<X~u??Q$GB}2?@U& z=s^!ybP^u4^K@XkILe5#vT;B2ZMAktI`f;bn`{$Ta&X)}t+a(s#p6p=`BZU){{4RH zKZPBE?{1W;$&4RlqWuFx>_kJFitG%(?S*IOYO)e7FBt~M>eLi%lO3FTN~5+eP;R*TX)KextsuFZ+T}YRSNnBE?kmc%scmd+V^jktd@Vb4zSUa#m(utF$Car<~ zaE5pFU)mq?i`I534@SPcQ+~L2uW#;!;)bt)za6RU6qilg50kZO_1+9Rin~lSk`W`(%uuW)JsOr_kY>H0Lk!#~gzv$m@^_@g3;3Gd zsDn?%W;Av6=ep_iPJ1yd)|+oR=s8o8P<|o35|f-x?NOpz!sJqw#*IO&NLLs#{Fq?w z4eI#hWbK;M7ccrFXG3z~w)}H1U}^@V-q|oio7h}P$_K|^e>@z(Q)kT*_Tk6 zcJ14z)wp&LyNN@${y79}h%Yy={47DOSD1~E0msB-gKLMp29L)1Lr?qRPbXH5$B!S| zY>!oXTR*uyLS-xL=So>-gHIZ0y7Ps@rPMSv!rXk za1-Ts#m_=|9-f0}!y>vp?$yMfKkvcFmoK1jxf3e9?OjmNLko*Y)S2M%KRn-CoG%nE z1-6V{Htrpp@|Wr1h=FcY1LOnnSV(4de)Wo@Ej=OOHOik_SnJ7;!S{_Rj*YaXd%*OB zB5zFr3Ip;K)~s2RqTl3Bba%z{F+PoffdO_11?1{L#b7dggnTAp5pZ_~vAyA55r22l zb7X|K!Gy>T5E1LL9_hoPGzWFuweQB*;*}+f@A0Z3_#6y`c8TS6SmC2#QMnK9;o`-M zwRv_nwsv+cAOcU64u(z??hqtgBXQ&B=JG@F{zyv4j{KCxWfXY78)F06c&gzAjEZ3# zJyP`F9XX=h>%h2vL6fuyBvb?$EyN}$YCiOx5bRE2EfBvVmJKOn&Br?o({A1vm%KOB zb-|?fI`Ktem0`wj_W0&I4l=EacXXRBKn#WGSdY7@dH)q2yAAE|<%&8m&CuB{!^l8% z6{3u`*huU4>D)euXC9-PpC{gQ_2rApAc%8IyzlC27UNMHo0~^ijCVA*wXxgI8F%;e zXr{a^c?Ev~whPge037OpX*dqLCFO->h5T15kdJlb|7ZgOAwnV|-hMYuEdZXW9LokD zKXVl+^Ab$!PiGIQ2lwES}vUFIjTMes0u&nYWJEJLF79w$G+gQM4!_spMq zcl$n)lI`>lSxo`DW96T*_N3#K4q!9$@xus{%51ufFC+;^iHeE}-|Es-lr#g(MrH7J zadOUd@`b;MEBam4r5}*NEw@vthWas0G!I*<%>||u2J_&XnpZ^VevaH+Q`n?<_gCMB z^Cz_f`g{I9OzzJVjcDZi(QcUWOAn@fqQh`Ytb(;*W8{{SL;|v*QLrGWKgS7uJr%riAtH zhdt?X^zFpK`)q42RMgFLTV!#vP6xMJar(GVD}CzIq`Gu{9T#VaXxE<~uhgv=-*|uO zF|*|iF`xX>dgZ2-He-ZYyi;hZZ=kM4KL<0tq3Z17>|CxHqhH`4&jQy3Obp$bqB9(^ z{b4bk<_nxx-&*wy>TD_arPZ;OlD9C5(`PKOKV+20Pc%F0oV59Q{+MlgLsd@%^E}?f z^@XHHjlEK(7Eu~ux?a|(Vff*gtSZ6%yo@&!M?<8YAL^?*y zD>2A6%A0qBPj+Eubn(()Z+|~1IuU8P|5+Z6yH>8X8Y=qKy_V2F?mkwEy)G#@1_adADFlR6B2@( zZ}9T2~Cn-h**H*d6b`_4XA zi621zV_SB#jDh0;65K&VBiLA8)bC8?2=}4JJ&1r@eDdVUo$$cCzx)jHTTgws*xda5 za$w|1+FKA1V1%HGlnCms;_8&)c?JqyFK`s^`Yl)zvMbG|r$&T{N>9N5XT+X!^|-LKXr4f>a?#uYXZUDz|gS5`^+)g@3`oBJa6Jt zz>CkpMz5P+UmlFr4qLA%ahu>g8cFPOEWAizw>Kg&%klJyA`kT3*I3w01*$Q#C`!Xa zxpPMe@9;Z=6vWI-A6-hVR>gMbUD0>%X?!}rXd5pPn4ToQm^afPZ(YeN zor0@lzeI6-t+V*hA#wrQZMZ=g<3x;&N9_50ycR7OoIw50Uv5h zci0%!o379cxMWnuJFI^eF8JUfftA3U04v+6sHn)k0x?r*?bYAk&-UQw#dDAY5Jbkg zV{Y6Slv7YpXWdVp%qN-HgXEAKC-CrY70UOp$BXYEyLY0M)?3z@hj3rPH=U46%Ry)0O2J5+Vuy1#jNE?#Lnrbbp0f z;3G-Zm;lT89;!JL@ODGLGzywfeW`cVjjmw@I3GzKLro*2l_a}3B#@XSs$YT z>7&%AD0od_bLv5wD9=M|g!u6sn=Vu)`V+QRqDvgaBqAk%*@=}FSck2(Yv<0#aKW+N zA3~J!o(A&*#^;c%tgMo3Z$3SO9Sda;T+Yd zM^D4$NPL}_2FE3ZaKS!KX?m$y;tkjLW_JDq{csvouFWhPPBOushr*ci=PQ-;=ldPC z%kkc!2=XKb6M+b+4X$btq#GZ*3Mu!s(x0WtF;cbhs`+fHJC3c~giJO7{||}wW+Tm> z$jixiU~Hp`3i7v8R`7R013dZr*9_Hp`O?Y>+TenMOK={DtE+!(dHGl~c9Gs$A(lW{ zIYq?=0A^zLQAx%MiiDlLe6!=#7u)+_c^B32BN!5z0J+xymguiz@||%=%|ZL-QC8O1 zcw0@5$Rvpa1O5HNkgOh>n+u25k&*y~xKADT%9yFJXH_5gbbqlT2)`I%WRf^FY_ho*rS9r)?$D5De>GZuD1D%zTl^t zt;RT?qy|}iYjgk2p59_7;1-bkxo1VBHhKI?!!tVoyR4^IPMgQMATW4F2iWFKP;O&r z>CrN}tR68xe=2NPM@3mIIPZ@@Q_)(xC!VkP?i5gqU!`u7e_K){2FIOLz|+u6w=+1| zJI&nVW)$j8r{RV3(SQ9Wwxmqhb;v{i?}Ii`j+hY&|EHg}4D#dz#bmsdO51comN#~{ z@uteWo;Pu)Ec$5fed!4Nz15NFh`~ah`kkp3(_^vXR+gr#l7(be4tMg+OCk|EjD#3V{No0EdfQ~~}wjbd^D0Re#U1!+8d^7Io_ zY^~LNyG2f@SSVD;0c<~_{gPB78v;^C)-}wU#~&M~4NvM|3uC+93(K+gkID06djXIb8YFCx8}Slwr5Pa^DMV<6F9jchoNzC{>Y zRwkzQoEnB{4BPx$M@L?B#aBd-|6=x%7czzi3#BO2`DGk#MO@veybf@igeCET2O|wM zKts@hH$M9^!hMvR`v63fzU1ue?3O$VfVR8V?lk-M0g{xKm3;vPK}a(w+h*MUHuJ4E(AJ?1QdUf5wgSzbkMEeX!^9`(bg<+mM#9H+uu3>Nv?E7_5(T^|!zB~^K6|5SA9%4xLdxj`8Af*0Bj7> zFj>P-+MROZBKnO2CLMqztAqI~Anoa($o{nGDY?Zf|n@XrpanlGI|C`dj%FH@7^!&6vvg;GizHe`b!W zhsOZ+=(5=2kQonvUm>U$Z+&?=xu{o{CM-hOpIhz+{6KA24B``{2^&~I2;Hhf_TfJK zf~!mH#ITPxV*;I39hQA=hYBac5A|6#ekDG*p-@`=X!>pHP5mP-8jAsK>l@9xN>jHs z9?lkeVI7z9X|z&u)1D|`0PwfKGk0v^L7vMtc+@w zAknkM2wDBzAe`4~vW-TOc|0|`()SL%(#m?hkPMDciO8TJV%z+3b^M~4k+7N#l7LLPK&to8FromZ6*KAo{cMc64IvWYK0T` zA7~%a8R;qQ$X@#fcDQ_%>R*|Z<{2%DxALFzt8Lo7zbF(60u1~0qx=~t-bDb_B>dnX;jA8oT`MX$;J_E4CM)lBjq|OG^Mq6=E+SQaiu} zKm$u-mRMv1D6nrxkrWMLMkw@gq+?iESjoA^bCOdCPpqu91{u+U!|mBz$d9MtaYa$~ z4sMePAZzdW{T4MfAc3!Cn)KcJB#jg8x?m0kcP;u;HCBe^ko&Y+Vgl=QmUn+jdkfkR*i1i5hS5g5oOrl&Q*qak;aS$P;95k7+a z7a@IZoz^M~!??p-*9)8eBjUL)(Nw#>01&Vm#y}l((9etfekVuA2~y{Sm;2S-{Zu7B63@ztGuaFk6X5q_;RhJq$CD-%w5}HB9=4Eqo5z~J)Gf?WJIm$%8jqvi`YMe?tq%Lkz*nv zEt1(NKI_tIapd`fZsNb?!ozyU2GQE7`XGza-`d7qU)3>&Z{51WYS!LJ_Kapds}t{! zJ0IY&Z?^IBjAc~U{}cfNh@zpxcgQTz=8djmJbQ$pCf{Ku-R|M*4i1tLhMB@O^O4OK=Yv?T*wW8Ns=b-|>u8%f z$k24>CjWwfR7dF4F_-oMn1bj`h9bB^Cu2c=jG9=8TEB;m9e^n~Ned9k~ zz1q4Lhd%LGR|^&|@jv94&TZhC{y~##nf8=l<10(u;)^@)#45XztMeeX2In19KoLdF1HFo+kx7m zK==W3a*jHAGFIPdDLzQUOQos!Pz0G_4jeaOVt_IGFe_4hq&boJgUCD`Y<(mdVYFFz zhkV@h8Ai>sv3Ep8H;J3hWn^bJfS^Ddge0gSm#S1njwOw{XQD`d&KU59Ac8)O?yjzh zo$-~+2i%0g?o`=}Qr`3aA1!)5UDsi+F9E4?e*acEIXO6-#MFkvw@V*AH)i_VEdNayy*VZ0Tw_?!m2n`Lz9=Yvg1z)H!_OQB! z#yEHScf2`VIH%B&iWZBvfWv`Ru^S^~Fr(%eY2swY9E*Qy1c>Edzg{647O<_)<4!Or zWaiU}6VK2E1?%l$IPsV-|17xu`5@ewoKNK&BC^Tu0$^& z4vvG{&~*l{K7&tmc`DPP-ERs z4w?Zjxx-K*YU8kQ=I8kGr7O5lAVpDZPu#KcXm3OYzC zhnR*AD_hz174u%b>utsyC}a9jHNMF`6#JyN=#+3dH;9yGdaoyhor$K2%eGH#bI@miBtMy z5GwU)KTL%v%r)RDfyLMr$km6N5XJD&?mM(r>E#)DwBDk5G;`o5AD;Oc1_nwj+2k4l z!c?P&oV86jzG)9Ol$KUR%OoW=11n|#5a2<81zgc+NWJAGC@LY*j|?eo{eqA~Fhj2D z575&p0XB1E;tpIWUm6;`K(g$ME*-*I*bF~bwa2S_44760-Sey27Pqo%ecM0W>=zTB zRNSmtw)wVsH{<;Ay_Co3zxt_OX?^*+C@fKPqma;o=*gt58i975+SQfw%jZ=^8d%vy z1O8&B0_~20pRCp=U0?+u#?!VL5WdNv`SEP8sH-3E=;(k- z+mN;l(gB;?A68{PK0be*M|w{Nwh!g7OwK-pkZYSfgDzHQ2%L+vdJCRS)Q;M1D$;aekk&wok&z0d>G9#iF3hS8L+wPNgx*Rp;D|OJXtu=E zk&Ho#WJSoOhhe}a^dYbcl;#UO83|)YNU+}C`>4~bDgtZM z+y40l>570``xfL_3_6&KeJ3r=OPQWmy(jZfj0|Il-AusfLfjQcN>U{j%zsSiG*mMb z0{u*yq+MxPv|xq(9;X~gsK&_&HL1zHvmKJKp|+$O(Gee(0Vli7i~+6n?+?eP9F`_Z zNd5(@1?O}v1}tFp8MX-+G_w*<2&`+5QPv_(rSl6@taOYj!-I{Jp^uTKanLwl>go~; zdIJ@ewaFI&OUDM|D+qdB;M+Wrg{8XTSHp1-%-g)=ues3joJsi)FMv8*z(wo7XX)(z z?m~FCw6?B?i3?fJ%LqYw=I>~m9~2%UQDZ$o2!se+m$tSd_bUwlY8MWUSMLr!ZZf_w zl3Zg@p{m>j;|MQXblS;77mbhN0vU}PK&kdVfUW)zTy)%+gHYXm&gJDx@100O_s|8u zL5qL+?ux{6B!u+?s;a8w@bGk3FbwH~57I3#&PoKGUzwSWU;Sh-yb2S`@*{CQ#IX#z z_`>eK?FH2Yb=`4S(*rS<8 zTW;*^zfVG2k0?*U+SzCFvQ$tUdv(lw*S`&oza|(V1=b1unbweR?B_S@HG?D~?eDzgeR>jYKB z=o32Kd+PYoC;l4xoj83e&{N<3<5c&!)ocNED+`}Vo7Iw+qmvUGbbZ3k05Z?7ERGvx zj)Q3V{Ue$NBovvof_t}@F$4}T*g&s>e+4q~7+r=@lpSeRaG+r9JpLRBvWjo%Pay`u ztUjP2+r*blC;$WhsAN(x>HZcn3gSqGv4{uEkdn<%z42Y&Gv&l)W8S>yu4N%gdGS^|h75S+ zo2q2rkmdl%lE@eUgv7rI%pGX7h^kc*;|+CNl1`lve)S6p$DuW`bo`%N{avMfc;$LF zYKgAh4#|C!eIGX|x_Iw}sCN(gHfeE!4JZ+ETU#0cd$8S%JPaw$Q?w`|pu}AifTprgev4;odypyz395iV^E4&pKJ2!qL z<7-~S!`@y6-67ex+HH3uQoI6=l!VSND6Q4|k9v4I7g*Lr-Z`qmPbs*|x{; z=|Wq~PQH82LG`18hds?s^Ro>vjl!Ap^es#+rk)ypZ;aEp>aumY7x?Ak3hdL%7q5u( z)WqU}dJrdPx?S$-7DW0FvVpA)m54NoJ>)0`8%yZo5 zuBysuCe9!%>w|YqH1wMEmUj9U1s+qoe}8~&q0p;x@NR{#h;MOu3UmIi{(D-&9z!QJ zd;e%%-yazlzB9!4Cbi_5(~sgrnM1w~Ppg=B7j1!Kg4lgPhoJ!zV}-Qfdw~WGypliH zO+_AXLxfen2O0U8W~hj?Dk?24{b3$9CkO#9#p++GtG7P}WhRxVE(Aa^??*Q~o@m^4 z1$qvJKaPNa-7;~*hnuGx+L_+IyuJq9F2+9*hZHis+uAmCPR=#uuN0FGIP+13JVR(u z;=z0Y*dS~$w-E*z8z)L3m4UQGW9^f{i}21JLTTj7g3kRLUg*bmt$uN;l+@d0Wol@4 z!fXaF2n$1tyF7Q7=%r3Vxo&{e33m`gB-#C#RRW!P08;`BflA<J*HMb6@82xP;x#UGrzoerBeC3uz-7y_gi!h(!Xh{aIa<)uq|p)2i2c<7XgIl9>&} zFM;`k2*~fzj79-bpBx@Rd|5=1N7sr!`t7iF!qDn#k=c7n?KUU5nl-|GpaN^$nDtOVYwE!eTe{0S zW<0L>8P}h=JTQwJFZP++x^FbEz%*#7nRCp3-*oiV+KICgj*)NP1{8L)oMVf(U`|_J zwCDLgeZMT&m12bdNQup+_&@6DLk0)hp37X|_n$0dIjP;dS3%b_EhrTGw@Tw}LT_1? zt;xxECusIejZWE2rLAR83l3MYl3iQUQS+;x`l3mfaL4eD7gN(MmcD%Qe&qWF1XTon zQNu{@d`ZK@qpxiFvd(KNjZiKhE~UCIm_JCfz`-mszR5OWZL%P*NX0=~1KNh?9yq?8 zhud|uBvkiK_EwEnnFw=k8Ufn&{jx7|Hg>$x*PKZ+%~)Z1sn2#$E^~GF#(UE*H`)q} ztw8CtiiP_mmNV@Y{ z3m~kX!L&){(&Q~oih-<#>o|e=a?tD71*SFpBJg6uPoWK)+F?QaC3hb|D76q(iJhZ0 z&9DLkPMlp_vQB6!Vhk|;!8kD&#O0QH?fdsBtb&rjm(*eGACNm?U!l>jy|=ouOnMd0 zgxJ18%AQ2>Zb;M+LCL9GAqXOetHbgc`XK7?0dS`X>FF^TYxKkCI85p#IFvPjSc}gc zDj?)VgcwO6tUt)*~;UO|)KCbKmuMxDVTv{vi6xYF0Q zfz$%k5QZKN+!@jv3cO`7@E05pvi2b|;sl1s)rF>f(nEnEO${N|;wCn6>53Q-=~d<( zpUW`xGZPyNzC=%!d>E_F!_(L{45uGK8yF@FcjxsFhK<rjN21YiGZk+<&5v`qukT=LLG>;P#|&v00}Q^@F?15QlZ?7U`P^EM0N3j4pk~+z z`SAcINGNBSsbPXin*6DJmDVYEGl)3u;NTE!*}WxiWdW_e@QX${iDAVJYzC1j3A!c z^>9PH0-0KmMFC_a^{e?AvH>)aftnDOgqznMAQWbp^igiwK}$yS5_<`oW_fLlJbRnf zt+8#Zy(?6`?G}3AC~;5cX~j1Efv<JeoE}alb?k&yC=?RN;Wxl08qS@wa|W;&(j>R?KvO; zIIRhj)WVjI()>2ru6(r{MPc{X-TKky{LN?TfkNZt!w+x$ybOi44C!c&d$Cd3s$5gw zC#WQ%xx@TJnCji_5_|YpR99A6_`-r54Me zuFY1Pk#s0l&$>BnoNrgv`~E&$p@&zH=WRoVT}!4+^855Z%&*9pI5`e*1x0*P)m`As zeKT5h4r&v%%bndJ=DmphIsN(+!AjarxtWkXO&M0W<*wd;VCuybevnK~>|T*BNS zvK&DHpMZH8d|CqFg8f|ACWkuM~8dt z4+bDFMsyaBCjv4MT9uOWINk~!N@_+%4kG0q30WK?{|x35>@o7GyLYQ+-?N8Xm>a%sXkn!AhFF+zzwC`| ziN#?3DZ=uRsg)R3rJH?}<+H(o0|y9yg5I?RhgEwFDZ7qcPWfHc%q%5%%kkCyKtUg{ z_(>8#QEtC5cqR_h*#N#NZ$kK=q4Bvj=B*gS3);96fdVvf`e-&L+Tn&!7NFII@LSL$ z`AA>EEDq5>$|2s733tQ>h3#;uZuBc?0eaOuJ}iH9Py4du&3$Yxk^XL# z8)beB23mNc6Va-UP3{BKpvb*i}kpj_M~E??PreVs15oG{|tN;U2E8p`eGzqbKawy)SFqlAdBz*azmbIiT6r zu3I<7U;i;d|A}h0?K~Sg?~3D0J$bvD)}dcX{|P5Kc?A@OFMwdrz(5I@M_a|V@w=yT zcdXrsPXf5Zd;(r*92??#)}SJ+MEkh}ktJkezrvIur^z_-&L7qkaBW0cz!TGeJ#s^w zCcM7>7xeeR)L&FwJb{6Lt!dPc1XPhm;g~&&!bcOY1CJVA@Gv|q41)lLwY0u0iXWG% zjct7j$)sy|xB`7bqM*mgOgi#;dbzna?vt|?EHkP{$AjS)-9z36UjX&+2~|3Z$<`54 z3?W|ug>F|onx3#y@kqq|uo^frGI8%dL?HcfGNufSD?#6QR)%(`ro?UM64pPk?bm&yavXqpx{2a;nU(H~ z9GiC~E8}N9mOF2EY~6l>v+rB-{wyvgF>z7Os%V>E{r-NDWjgmtoOKrBE`E!;AecCK zeZB8tMww`8o8%`#lHa=&uAKJb42l?cp_}`gZ^b2!qf960eOBknebUbG%s1B9mr`v% zbhgi|XYR8lPuJKx2ea1kvO`DDoeGq4*BTM$vd_V&uOG?OFGb7Mzr<0cEgZ;PI8%B0 zS?rk0Flwug2Q#)d0>LRUIRirnA2)P;dL_5KM>2N7vv2X7=WE}aYEAlEbX%01Ykt{E zza7o>?`4WG;x6nlS}usbJitxohnGg_K`U6qOPl^KH7&_oD zewv^Wun2j(>>Zxh>73-|4gllC?&>Ak=A@mOUqv{6oZncdL@yQ_wx+~NWkI8IK#9x`iBeT zJD^uGTrJB?7dC38d-;eJCKy|y-bZi3@*5x^#CCEE1VU|0b|phL6;dZ@RY|(WiAMz> zPD@Km$++ro2ArLBfq9cn89y?U5k>gsemzT*-w9XrJ3)tFL<9;e4-uXEuq|llW5vN0 zfRmUMR$|d<=f|>2=6Zs;AOs0W6HS}^1oLC3Pdnkhv4US286DOAnQe(E7 zMnv{v2&8Zqwe9W$5Mz*O?M{(Lfc%11L>yLwVvJ0f0AqitW7|jYm$>zt#82xV-N zc2;V!6Oe;EI=?{ix~ayhKS0~qxHOw?d|+VH-uTsepm~f`Vhs@IfH%04KbbTjqOiu~ zO^+6XZ*YI1gQi~$0evThvwU9tX!AhojWS*7Lo|1iR7QqsBIwBFSf2mq z7USNJw5Gbo#4sUvUbLBs8M9j!{-Cd6nc2F_>y6-_jIr7aOZt8vyh8(C7Kt-t3m<%-vU;L5`81N^#sS})rS_5Q z;x(sF@HWY8I!`a3!P}-br+Vb(pn}_`&%GB+e3clxwgh-vSoB^IKUjP>K8JB?v9UjM zTcq4{pXO<^m@Ag*r{l*rYd&1%;fV3)3bcN3t4nN8)%cyj)J_$gVoVjY-;54;t);EH zE>=x`-)y{V9Yx6M{;D*M)sdEs=&>exbE<;j2{QUd?)@_xyG~V>Ek2%}mjR35y2GhJ z;mz~JRs&CT*RNj%P_T*F8LGX*qNgmL1zZdRimfFT717OHvdKwFq{kfyJY_?%si|pM zbsl^2H~gnKa21mMam@WD(@QW^zn!DmKC)sTdI#WFoMJvQh`K58q0Mq8=V`Ap83!z9 z^w_rPltcV&wO52q8IcUj(tzFB-Y&V?fRo1Ndp2?#5qBUvA!)Y>U zg3uiY)O8+gCqAMCtO|kV+qZ8gx+Lhvhx3z~q|XNt&4YhtKX}xolAj#;etXTwu)r8G zj!X@r8WHnhQf%?Lo6pJdcZ!bq3H7gJ@e#1)Z(aJyXIp*wys4;1Zzenn-~uE!P+r_(A@Di z&+BLQSn&$glFn5I+t`n?9v&DIb$f_Y??{A#q%268Ck*jmF$j|!P zY<~CUjk501Wk3I);d*&J3`|BMyeG`H*0s3{)eQ=;2ng;IGO<=Unx2);{v#*IY9PAe z)l7UP%dSthR;S&A-KK=A%s-D8FK&vzl_|cBPiaKX?hnJFy>nmTu5Bpd%946bxX(SK zq;;P!Hx=M<`(Ali@o}R8$2|&EOC|XNd^fVQG!2p?;ssS_o5Szt@rPV0u~zx%5>!@D zphewm!Q<}M>B#J)%WKwJM- z4Wf-E7CP=?EH1W{IF(NQI_do+7xeC6iwlsXD7T?k5Ia80|mCl<=K?=qo z4>cfnVzL0Y(fr`S-o|qvde&OinC9mF#N^DUqpQWY-%TyVwmL$4xgD!;0om{+MlKP& zj@fYYGw|?n-Bgqge8ku^H90*Uj`EZY-zb(?RBZB?(CTBXPJLY3ek)b)E({vFHo2&L z@EB|=mT#WkgENjRnAfEW8Rq+W{}m{mo?Kd3g)?!3?<1}N@%)=IBT4Np7-k3~(g_e| zwfDxV_72#pe3Ci^=fJuGR1ZWY0tzVSniIa}5}%Ov9vJJ=mvBEJ`_b{A4mj2CG&_64 zzDhS|XZk<>n?vg?8y2OD2lMx$$Hx!4@LrqM&IsmzWqc{zp04h0DV;VFzilVnP-;P& z3(1dp<*><7MbpoOV@hFM!GL$4Pz~2=h0Ufo)-&%f{(Hvt0V#!l9_4*!|&=rI+I6>hpPG;=g~tNY3>X zxLMJ4U%NLbKCU)ZO`gH}V*To<##Y+_k!s9_3)Q0Y%-Nya3ds28V=X)-X z^WeTdVZP;5Z3=n^Ch8tF1YP(TC(q*MeHkr3%rx)nu0 zI+sd!!?~t=fA9D1@0@YQKmPHLaSmf|w;Oe_*7M9~&ilUVwyIWIWprZr(B?vqb}gQB z%gTqt?Yll&9r)qmx8E@A=k#O#?(dN=+bgcb&-U%t{_V7xNppto20Es-I~Uca2M_X< z8VV#^w=K^fkSTfe&iI#Ft$*38c+DECB`$%`YBFpFupp`Eh}8&%u)&Uy9L&kVQMsz8 zy37S>H=t0&h>;cW90iafkBW4|B1LAo+)hvsc>qRL!5_)+D0JydFQb;MiDMndSqg+H z;6Xv2qt>N?ScI?L73vo#A(>cuBOr2ygv?~mEh0HHGyB+XgsO3dFu&Pf$DoC_0Q?cJ z@^(;vEIoySC%@t;FOIZ@3muC{fD1!Q(2y2j1^omXa2}OS->a*=QH8$8GXX3(S>qi{ z;i(066(=&4n*g&kLL36<1a$}c1UPs2ERJSEgb%%@n`R`pl~^zAzab&l<|Qkus;b2H zH|Qrn4hg}cr&oDfS`lN5KExR3mBMxVeghbO+{0pq~3#r$a zK3=i1(_Z#@r^17zl-a!QE{mHt2@AZuK#@VpNgaBs<}`BT0(T&4Ok5w*Izd$Rrx$X8 zbpyr|jISVICKv#-8mu~Oq$;6_uUji!dmq&+Vj zdjK))AXfO&<;)UIZnGw{UZ{>Rl#n!Rbzi^2lSyLKi{&)K#%F=`KR{4((mRup{%xPA z084e6_Blj*BfZfW(H}Gh8tKGe(0s;6VolP_6oDm5f>Q*l*pz#ZDl0{ zl~s8|8i}ihZ)Gb65CG>wm}AOW$H#~7*OcAQ;jnMte$JxL1LQA={rO>oh>Xp7Lt7tyeOiiuyb}N|rarIW<-Qkqnh~~F_yuL47w3U7| zOy>MbWAm=4i#p##JxnXYEq}y6XIguw!aao=-I6A$^-o6|uC1%w@%#pjd6wu%-Y?O} z3sVQ2UrwD(ZwjG%WTNfo_ijC(r-A5pwU*(M*Wcbyawn6Ho-S=z*SFP0K^j$zfKhfDSX7uL8``OfCh(03TS4h z`Vq{8`x=ENIJnw~sQZp>lqy0N5D-^R^y?&I2WzJN=0qDAFbUa%eUIO%*RT+>JC7Vq zuCPPt-d7u`gpq0x41i(T0CDYm%qF;2t#b3VoAu`23G_(?aE#z_l zX1kBDoW~%S7~c-ygfT`%LX=tvAI-_l1&C&xT*ss=3hC&4ECVFbXq?+5jR*S`@lAsI{pS>hFu z?U32bBM6My%VmKI>Qa&wPSTz635kj8xqSmQ6;Rju3%|yK4#5!#xCxqZ64=&P6MnR* zw?vDrl8AQf;x}4hOTgV?U(+}#XFJ<7(Ld49; zEbVB7A`t?TP0F#Q_d;d(gwkOCk+xfJp{&HmI0bw;7>SK%gU;uLbWP~4yO{6lLWX(3 zVP6gNIyUGY;Bi2QUe56o;-?kTbdn|DfRSrSDJfA{gntBjF0i{IN7FAn{4Ba4bT-Aj zwL6wCxrc>mz5Vi}8EJHhY%{r%y8%G&OuIo&PRpfU5GYBvb!8ZPcq z{iibQ_^W&8|9}4-!DY-tFaCcP#qAKe`us@3gBDBB9O;vJ|J04$3JPMT>dV%m-#qI< z9N(>ccmDCmr#+EovGV7(%zs`u$gzB=|L1oe{eS(P>EH)N@!8rd{}w5d-^|`+D9gQX zDpMk`nDSXh>a1l>=J2{<)(n*%SJkaT+m=VwoBLhgDq8J7Jn0j3$~6)X9Er0b+X?!P z86+31$N*=7VOsk}67krHRZ|EP46c-v3`JX&NN!Sd8t^hAw*>kA6Pqvm8;8(BQK19i zC%Yfv;r}r9Bn1|t6NF4J6-6ymH2y7rYvptQ$0c8=-uKvPV|iH(2M^b@8S^o--H3oB zdH{fa_@TupbsXSlMI%P0Dwwop)lZpY`NPpF;5;)zO|sF6kbCUgMWs;$vXTrmm3Xy} zkkwHA8CJYugJE}xHy9m1cx_VEPLBvg_E;f}+Nlpbz?4lcaWl%p-&zyM6HG0|rglA>hfu z;lc)nmmXVOml^Y!zaOlj&3!LzXMSnyf18(8(L1A1k<~PM^X=H-pQ>I^h`r5UOL?2( z358f@{`(!v*R><+6pd^b87k9XoC!X=ny<*qJLUDW?i)+b-PhxF{-u^k z_g8FplaDmr=8-9blq@*yS`#O2{byQ0t(+=L3cj#E4t{{>BYDL;;lr3E&}-AAslWvWeIFUteQls#kZ3 zakqYoH)XcZQr1bDm`^g$Y294%qjOSi(Nvrg^;P{xA}22KlKHS9(?DL)+ZFXx;#Pv5 z4UWpO=Q&jIZAK#(SHI=M=l!1wuM1t?Q9U!oxZb&Mp*6_DPBhu^sx}0IRV}tF z?6`!(e+TMW#a85u{F3mFAg_^EyOP;Xe~CjXE3&Ki#- z4j$#$NK1ao(~Yms;}Y9vu^|TPgwW3qj#bf)~PnKz8&Jhr4+Fj_1J>bh$ zju}xS%=Iolb~3;}K;VP`BBUM_U;$2U?zOiNK)4!u7fk9?gFE8!v}Pp2TY+}%K^ulq zkc5Lrdmfs&1cUGil*x$u*GTTy>Y0A8@s3ClPyiCD%-44#vr$*OYy`GO8hxxWC^8XB>OM}FfBe;# zC_k~ct9w^{!uyHUbWyq0 zi5-~-lr8ZA7>)79h8VxXLC8lOTUb0s}U=z~z zb`5VU$4po9&P9+0 zWRYoC_<_`9lq^mfBBQ|ZMy@8PFU!f_%9?X{I7x^bG>t^Q3$tm`;syeko3!O@yM=ZV zwaeA@U9vhBQU?zP!omQR8VTVC^S0{u2-tamI!xI$Czc+-hJy=EbS2~s05v125uZPQ z&ej(B`+>=%Wq|%dDWzlZ+44z;L93cS`iqLES;_A{_-_O2@MU@Z=d435!MBY>Zu#eO zxA(V%7VV6>I9BJgFe2ijvUEH$P$pS_Ss4;6)A_@Wkwk8OSSwY;%*Vmu*`w9A%dDlh zrd^E&+=ZPTMYhZPFhT;5+f>k>8@DwgxcJ*U%C?cVGt|DtRF8wrUc`rL=Ltb=pRy{- zT$#(3lO{UMj8(T+$R|UR5@|W@G`9MIQtG*iFOR~@E%tPTtce*NkI6RmDqeQipA z?PbwzCN`?9CjzUf0xXlW-*pFm0z>bKD?17!cGvK5EiMldbB^VyXF)|t^3{lhrfdPS zrcbDi-yqF}kdailme}tJ2Z$7=Yu{dPMhVEVcW+OO1EkcC0C2yDgbQL9X%ZhsG?c^# zU3GN%Oy1&@CSpO6^1=+o0p<)gK7F#3TFT}`M+Lqe`fQ{1@c@&8uslbQ%6UaJln4wF ztwg3G&DLS07;%Lmt2N5BWDr0HhzS8eQ$_z$d_#6AVK>AIay3lc@(zes2*;GLrJD+0 zcvyHn6$#fwl?i2K0H5|VpcKrKE~tE==EcFW7RwfUzBpWY$#pT1g<*;jKHvjTgFs5? z*rmgeQPemE-`_g?f07kJp`3!qVX_x`%hT70H&G>R;>X9(Z}} zSD6L80+$-XR@dJBcmz+4*<9{0`U>3<^`6>*r%zYkf(udCd2|)Y`+_?;?H?|ur3bQ6 zmJxhL`mS4dwOH(Z^8H{dckfkBA@0hlu;v+&_1l^>MtXHu3a>rj@N#wA@_I+<$)Q@RR|b8L z1N|%G*_4xeG-j9dg%VBjk95l=5S74eXvoLSp6Vas^gz5%Guh~$vh@jk^(f~SPY%&<#Bc$h^>yLxYovAOiK-?ky=Jyn_YBjT8Fy)_R6C0xb{zVs^~gICt$@Z`XUZ0|fROXv=M8dcFe!e;jD3 zgVxYDKx7PQ$EqOK$lw$_#;5I=IeVZEG{Lh46RfPT#+z zB!kew`G=XUc>XMCt8#9b{c`^e1B#l2121_Q*T6(t@(B+~tjFu7wx1NZqPY)(H%WiL z1M+;TUBc_D?CHJFP>SjUc2`bm7E9@HYbhwukrEmpAqLbWfvSPx18GP3rUX&Ah$@)+ z8Z?&(4v?XON`n-*AOH}*vL4$JfKw>BRuL^xx>+?DEd`7me~Y+Lh>oX{AL31{&hd$_ zN;K#45Cx=@3;PYJpW!B1yf$gUg{`J{eU`?%@6<;dU)EulqR6u&K4uq-iBO1ZC<7HN zr`XC$A!fWjpN_$%nA&Pq**%w{vjS9%*|XYr7?t`6st5c(S%7K|v%Nvd8sp1HvEZqP zR}g*cg=jgV>4me6%vxACfSK;*&!*<4^AHx=Y-x(U4OE2e-3xycrL(`uaBvK7&T1@m zmAd7eGM0(aJ($Hn^l(^&*hCp{q;lJ~?RHxY&oYEc;%-YhgSnW;jy>L%S)Sv+n!?@F zUa~Zq(X7=~4)I7VFRGg!+=@FKwykwx-=KlfX=~lzGvdOX-6?dg%f=vtgtL@02dTq6 z6|S`TJUr4i@a>wf%3mCXpFY>evW#|eSBQsh>(=!{v#+u?+wssf-jk zDwm-(R~CNz-*j_hZ)caMRP!nCTg<+8_uVXgn}*Gui*IT+yYQ#JJL2f$eM;s>?MU>O zo93)?{$BU}pBD!X4i0Huv@cIM?&y*cdC%HGf$Tz zAGBi6l?q#JxX6+NE9L}$rWA%L@Exu&9b96NZjP`h z&eE1UQ={s|451NGx8V3ES6)90b5CuwrB!Rnq8X%7vMJ}vaos-eneNBxLJ6v5AF_pU zuFFp>p*0-q^kBDw^nnVj*et4?q$U$GUOP$e7@E?xM*vq~*5NVFer&HLSh`6X?@-ww zBLHz!mK@^i!Jh&q@7z6%WgH7G0Z*e|O4B;;?R?NLcgaoYR)KE>sY+Hqib+AbyNE-R zy~x8Hi~5`(ZD^R;LaW`;)OxVHc3>0Xkah!AcJAC~j@m&SNIm!#i+iWW)|sySNNUVN zFoxWiLpr7`QcoxZ%_j|UougzSnK+PKd_i&5uYK^p>(jNtR#x%uP4S(9yRA%lb@Rf& zb1?k)j<8^PpR|Qs+?y0AmSvlfyj&?fvF8%HZIx=`8dR3{fCc zsF0Aspw2ao8k8to@yCE4K!cFDHyr+gxXRAX&PX04gG+Ljm*y9+ao%r}K`0pT29G9Q zQIF^Ur3EmF0gsjNr3q&wF6mteW|1MRh=3IcQDQK?0YlW}vJCtX7ON0q<}exLbD2RX$>sgc0bCja4uIl~fIf{&(qQU^-|R;sk&W zUwimEP~d-km9?v(Bcu0taXjI{g84$o@uB+P2($-`_Yzm3A;;wJ)j-k(E*#yR-uNc) z#BsCZ_$(X;4>I1uN5>CA^*&{j5cprOivKN3wf=SDJpSfogtJ<{+rbJ(761h3hefK(ZMolNlf$E)GD zVU+K{Kk`u;!PQ0M>3 z8;ZhR`{!5LhLit$SIDd(oBGdtb~{x&XwT-Ky|%EL%1$6NL_lG6@W+)b4cT{gTNPTm zh=~^%1Php~?zvlT^r=kj=eH}fHVK{W7yG+Q6aRcHGkY)H<7Td2VoN%5+e`ob8OCTE z73dW>3rwijvvAUKt{1NoPp5G=^LS9^z!|1X=ia&SLtx0toXRv^sl3OpVVGhy+0SKuKRjY%7Q>w`;XwCN3x{Lb4*6hQJat zvx|$;WZgDAtWLVpF)-+*m*FZ(M{D8wOz7sGA~SQG2EViZW}~m1nDKbFA7MU5-JGI`9va) zV8FqK@B&9E0vvb}rf25ns0nvY3=vQO{esAs?4dX%$V3MmvTf(iv;a-vREi*j%afZ2 z$>bbT1>=cpkjBAJK?nnM`h6RBMqyq#nK%Wni`Lugy24^st;c)r5D*o+E_0Jkwyh~~ z(^3&P70%-l5a(;xta%PD0j%?UoNp>zM~GOdUeBLH>INz@xd*M-{VCzlY7k$eUhTi& zrXe0F93B9Viy)A=S#Pr?NrH}*zH~N}y~BbfSiwCdHoKgv;loLlHfiN2^{hVm+vg=YoH_e>TJx3W06DOUte?YGML zV6tPm)&7aDqESb$ScMgZ%T`@joZRcG#k^PqSmxSnCbyKCkT~MrP?&a@0 zPaA#l5$~$AP7Xf%Az^D#epDDU^^GZx(Q+l@vZNpTbW0wtunVF*9b2M`ZwW8^wA1)Z zW!m|J-#k9P+kft0QEa2=nTjgoima^*j{9lYH?D7ecJJ%>mOZEVRi{Q@u|_Nz$h`Fs zu^eo+o6=d`rWqmmcJ&tj>xQ3xce9=?i!sXJ&x>HHSgk=tVPIijzUiJi|D-HMG9y=R z*RoICt@P~sLCowW&xW5v@I zd^rULH=}tkH~3^C+TGXZ9vZrXWWE6tLv7n==ZvJDJW7eACYv9uY#hIX$y^;amx=S_$B=pnOpnUT9s4e*<+b`0 z8eI;iW96hN6cAutO`np&oLzoz?mc8(Ybh>_TRE%f1tb)LO*pG&Da$t+#7ZVbjjCA4 zj2_6rhycf)5^iS?nlaHX_4x4wPo83aA}BjN`RyX`T$q_^w3KJD=XB>zw_SmJ776i> z!7Yx15_*r+6S*M1YB1DA=%r_{L32Z9Y@R#G1QmQ7kEa%EF@J-57qeezXqR`WYO-g` zBi0>>t`7HZPapD+ZuX58zCJ6+L}yaw(H`EXCMzoJ*ic+r_~}(@se^T|`J>?>g&(F5 z6|czpYKX*l)V@mPvbQNeA0u_aW4oTTf|k>rRn6TjKhiGvUGY($-4X55I%876`-;+N z4+%}clx3`T?qzqo*PkBq)7QOM*);C@DB4xR?Z@}^oSgL)MF&em7S5SoKQ)7uRhTc+ zs&r{^*ZzP}{kk7pok!H9$Zvnu*vG_6ehlOKoiciET}r;Mo=L;@OY6t=W)oKG=-dpJ zP!idf_C;s;H%o7&eX{+zJr~}OIZUO{Tv|KI_o0@Zx0y;u(=h-K)tI8M#(;0N!Ay_` zcXL_D#F9y~x~urJiU5xrTRR3z?$wt%m@l3o?=^$=##jr5R$U-TFS(Se(Lw00r__xv zNjWmvN|9H3>Y}RL-miGmG#g}E`D1g9UwG`rc;^{4sRKq^g+;@6OYlVUM_fN`lo0-6 zJ$>h(WpBhzO1!O(*3DpPJZ;Iw`<$)_Z_6Cm_m%zKvD%oaT=WjT&>iVxgC#H*7>L^*SS1klj*JJ437cwx=FKX3;YdOIshZVu^?L*RNm50vRvc5}*SvT!d5Nw4!1R+Np*1n@XZ_N;7Zw0YixhK|PoWVM4S(GBKOefFM$c z`nMOmU}`UlqFU%<;F0?a-93oJBox!|MB0Nq0uZ10wzL5O5yT;Bo-?`{t*tbgE9AZ1 zCG*qKKto^Flx@e4Hk>H`urF;Fv2f+WoQp9seFu>bK-jOsUI5d#5jat*;g z;d*a5=2=~If$%G5MX;kSh0;L)z%CWq>hkL9Xxu_j*l3zaq_{Xt*UB7LsBBEhc7}FK z`Kr@VF1A|;A|(M1Cw*AeyR52>Qu>F}cl~9O(uB63W2t5~Hl?@OGPpob)S^eyf@QV) zDppPk9XI{ivz)srj0!eJqU)A!lo-&aRNredD+x3Ir1gP4cA*$PMacW%__A!!wj0Hz zW^&BMmPv{M>_yPFt`SeH$Dm^`@1qd$mSDczNg!>Q@wyeceZ&lu5N-Q>rr+LTNpl&q zFOo6E9MCShOFF2q_kVGaV5KK?H3H<$0FMA&OgbFYxKD=QQAjiD(4?a3C^3$7If;yH zkQOgdmn5*P>Yzlrkul<=Lx&#=v8WYb80BCNr$XgPi^jg23}|(CA8WC${eqD~vXp!f znnZz4R3Pwjc>Yl6F}fK&v+6mpi9ez!Rc%)!WCAG{x!$8S_6~QAgh3$97h5&S&IGx~ zDI)UCCLX9~P=<3czgq;^p@XnPYbG~^nwEZDyLROHvAdh7hT^xEy^2V+uoF)1Jc7uY zH;PG-)*EV#HI!fVyV>~#{8|vXL_Zp%x60V+&B#aGyvvGI;w_P5ZP9fNL0T7~^q5|xs)V~(`n?FfUu zALZrg-IQM^aB9qxWqr`xEPx2dDuqD;S+V((dt`H72{>`tvMFDy_HWLxNY_Y6x@Etl5lj0~H6^atP2 zemk;%^Tq`lP3G;LYgZnMtLYss5TFCZMIw_Rqv*oA>%^jtfdUQr+m}-{)L}&gy09A) zIZ@Ph_Vtxq@s+YIAH$-Z#UzoYTqiMz6=IMJ&|=$cO29bm(m+YbzUIjlaT)ek%xn(? znK+I*CA*>#*f^d9qO zHpi}kC~d*CKS=o=LsN~hPh=k?aue9ff!%}dB+5j}A@^#dZoHaJ_%g;EYBdjMbki@5 zM=m4ZrWUqvA{J50bvy`X2YGvd3sdYz_5XMZWGO07Y*$exx|!W{jxWg(ByRuyiu$$D z`o&)0hmx_=K$%<(s10!omS=fYwm=oI2Lsu01TmlK(J_J0Fe5uVC}V)!BfyMA7BiH; z>=&Fa>7sFfkA?~sXBT_-v9vJ(9&k}+hG3<3vIFtlh9N=5M0+YmE zzsDJOqYy^7W5#Vm)xU+<`#m#g8o2>2Gf`|~(W|ACW26h?s15BzrNtzgH zULvc*okpYsO$U}g;l2TNV+~^*Xc=P^ROmjSmq*FAh?F4mkI**{?iW6E=ov769|Cuh zc?GEOg>VW=FE3a_veW;AA5Ro{f1ysD5#A3Nymgo2vF{Q95$D-aJaTH8RvggXL5Q&P z7L@tIuz;h+!Rbnen-zs!n`6J`Xl3e)nG-#I(e~YaDki(d(64Ng@s(}9eq!dy4eQ7L z!@J_Y-Ph5I&S@R767zrDnIkYYo^$R))5gJ!08b%u@Sy2!M2|L7B-qt4+E zs82q+FVb96K2}q9blU3aHNjA|R58DU^G__&gI}}tJ**xIWBXKCT~x%D8k|2#cZ`|s z6E?WS<~_f@i$D4>;H)b9;fOu=#x1qIjB&%m%|)&H`!B@;dpR-1lQ7V=C6ha3cBOr0 zrebYyPj=LXlOmasoETP>GSML=i5}~#_!WI(hi(^oRcm= z_ugvoe#N@{pTg(EW*!|@j78vAYj}dGZDB{k!zbUx{D~Y5VTR57mlpvlBI(d1widtxoHAj|y3WaEA6^E1#h~DZT0H4d z=aUPgWa=EGB4B#h(7m4g$w3&IiM><}>| zPbodBjNK0}4e1E%0CFz|A%rr3G7a;acRnos1h)#X_LH6!3JAab{p}50oa0{e zeDFa4J7Uh#(+C6gIoiIw6a?xcgaqcJdLSV}`0#QQu9oj9l1bZa2i$KLT3i;BG{48f zBCbGB@!r%*v)PZDlj6;oq6ZnGf9t(sy-y~>S9+R1+UD}?!&N5(+u_P0S)hAM&eE~z{dLPL=&jwCo#@(A`%XzZV%-he9B`*|)oU7B7@S`o_x${?*h z_V{y!UL@joz`64ZAF0K4e(;v?7a>4pD7FeTQ(-0Le z>BBLVPZsB;*1XNHy52a+a<7&FKIZ&|ZOG^%z8Hq8Sy43Qg6L<56zPM_ z?g!hCUemQx+}peGq9S0_rqtxleI>D?%GPZ;VJU}B-De8UT3Tt>u&>(v?wJ-F|-``wI3 zpVod(xt7_Y9Yy!uxzzAS(Ou{4qwe)5Zv~~yin7UltW6p5{@PU`8mds!vu~Ar*)jL} zSyhICVMRwp>F>@SZ#O&<8ecPES$Z?iTW9Fp{SrnRDr2EZ6AeDU+d8aD6?~SOrN#_P zxe;g6w{#7kH_>F;_p`OrNObc6MN#V{ANuIb-x>ubcNQM@X)D!cj-1`xLs1-DBhxmh z+?>igs>K#TshySGKHs+qxji~Moud3|d(wANPwcTR4I4eE&3^1q($LEXtRkJcPwNoNG$>38Vp*ov!iaw6G#zET# zW16tRuz-LbM+^sbf|~CPUsC>ef8PlV{fza9*f?SWH{ufPoK0d=Fp(Q4}ktr2#^qV zZUoMX5bZdGISD(S6cf=N;$aFn@=N(~JPKmiyp!^cnVsSmw}7SvqsbS(`SYAKATH9! zlSK{t#KD8=Yn#Sd8+Wc24W;qGe~K*>FS^svK6|Qge4X`KJNMgYyFAtwntd{%VKq`n zlP`Tf)_56B1=NxmY0CU!W1o870gQ%~(FFw54QFN-k^qk=9L(*(aF{a{Y!cWWW zmUDRDzkmPZfRNgcDIqP&xgv)0PUj^*`Th!36nt&CYHVzz5JFyO|wrv%KPwBz})WbnZk8T0aMsb zggX1Ly+BLD884xjay|MD)9=Ff;iG$v`>dS;kEF5YZMxaWx>|-+KI*)K+nvRhYv*br zH|BfmSzJ1Sr{3OeoZCH>=Wwc}rJ+-1sHL6oXno>ZY(v3}`J5I*&QDUVKiH@&>pauZ z{1wy@wMX~+MytK|Yc_;SxemW&t|FiMg~CC(`7Gh-c{S0UihbR#8&Lj~r$~s0>qndX z`ot`d&tc>rcq8_x1m$K0`S%FQg!(#U^B;{9TSjrFWJ&DUTgiS{)K8uQmurFX|^dRgd)D2Luq1C1HBV*-oY;B502x zP985s%m9FH;?QB9`XgTmQbN9U!CtYvm%zL986;;B}l?~ z^HqBk`Hh4^_#~7SfJDO{$t_axfyCbNX(|;8XsxXRC&HNZ@mzrsk9vA+W&NNWI5KhT zK011jsMGUMXl{p0;=+X|{hSrU=8p*YQpua1WY&7fhgCurh_JR;2>MZk>hZrB7+2eJB#3nB{BAoyIPn^(ap$;57)~+}HCZVNa<@+d6qGxBV}-a=K689_ zI^hS!d;C?$?=l0S z?t**U1SA!LH*7VtwGdZm}YR82g)Bozz3jsqfH{TEVn14LG5hIJ)$ zC*DV+h|Pm9Bxx-@y-`ELdOJ&;0umCLpKa_&XG8J>;L2(Q42SNL=hAd#u@W=S10Fsj z$P6$OfGBmaI>_V(087}KC~?ijmbdj-?njW>u@ExsPMLU67Wik+L7>_fS?p zA2#hqG;pvSka;TzBrL6Wc!kP_u$;$Fogz}WE-#e@EAPMATgDxKq3#z<)7+3Zckp)2 z@ZzU|VSW2tc^f&0?9t;7t-g7VT=~p3?{Kv*k;VOEzjmCq%_{ov>Ym9&pA|J5o6F|k z`dv%?a^t(1qgzJkE?v6_Q*BT50Ws!>ipPevpe|H6J=lL(^kJ-HFo^r`<&D< zu|xUk{y>d{@|@1f%`?yo#LvkcWSp|1O@JnUztPCrxGul!Sr z$gj-$LR1*lGrA9KRBS)te3H+*#y4pd`Gks`n!gs>6qm9`_wvSR|8!_hS6AQubYn+I zE&W)*hj6kNXlmck7YuzsH}ML@Ktx!2V%h7T^-lgXGo|Z2zs3C}ZE`rE3Y_-)UW;|+ zGHNvZcGk(J4&1&in|~edYst+SDL#7rl-5tfme=}JgZ=kMem~iBSkRaqrIZDSgZzAk z;Lyis@jv`e2Tj`wnRNa&WX^JOey-+nvf(?Sc$vpV0XAH6V}e_2F(KO-9+sC6MyyI&*H zyF-iSAV#?X2_35vuKkSsVgS^6vNp3&!eD6RPQ!|T6=4^30S!I{brS$1F~t6Jss}3# za~G$YS*4v>;30>dTpfBM=x%sFDE(_FQo`o&RWMi}RnPh?BkF7E2Jc%BG0WhOKC2mv zuLAvAS@M;d;b=7Wd9WI6;m})LT!g%R1C*Lv5)#`A1s-^N^S;lk57bI9Q?&w`3o#dB zIJ0gH_TD)NrTlvAU;L=wI3UXeOht^JsGHXXenA;TJk;nN*G+!N@Ai=dpo|)U7HWL} zHpl>60K&i#N30X*(rsLZV0wN~X!(L!4TT<=a)R+zuo01Alnpmjza{{=$&b6%cRYc( zMBvxHc=h^`TR6wj{C~s`5um#Z>Wl2sFgQEsGN9aq2Sg1B*o#59{<;Ct*hdlaW+%jDB*xVY4KY7G0qLBL_m(jr`8+)RaTRl1);-(B$LJnu(G;#b=|gjB zYWCDqnp2__c6Z#|QD3*PW$Ci45h=YM@v!!tlze&C%&cUq+->f+tBv-68G)|?wnO!o zGp}C>mi02&?R4WMN+sMTYw!kig+$qw&_4q-gZNl+?{R&IwH+-87S3szIQAm=Xc)V;faRj!6F`g0%dRr-~qWpTbTHZ+pJah$TSlQ;IFo zx7r9w{xK5hf!0yBR2{lOL$m3;^|X&w9W`KfV7V>1Q~5Ti_kCj;9;^StQGekjg*Pt9sP*&u6|zZ@l6!{vpKY2N9k~nw zS-ZBHGq{N&t8ZLIQ_t^2|L!{Nm$<3uSg*mG^83QgxVHz}Tw{<6I$1O58o?w=$D>vH zCH&#{(XMF*Rna3JbAR1@&2ClR@QM0x@aL#L4qfZ?VEeHltHTa`n<_O&X zT=&siOJ;d#x~KnIra|beeegrm%Hq4*86Ag=L`T0#^*a2H7`tWt?AMjGHQW(r3sTP*&tY&3}dG!Cns&*otSeX&jUgFW}E<1_&n|zs~W|7h^Bqr@GWg2)cMpnH9vVew#BwB+O z$;-UKU!A;yhIF&iD*B~sWVUC}}CNoRh6LNn%ItCDyLT0iyyR`;+YeRAK z=nY7U9T@czyb-J>)H9K?hGc>U41rMI>FjeD!uU@dYubkBWX@yq0jQ~SFRjTOEO1dh z_?R3RO^wx=@>xO(>K-V&NdZJSE$D`T0o=^hsk6nDab6Ld)G}Q?z~BI0H?y#)U3Eg| zO=1p0kXZ|ukIcNpqXO!xTz0;fLfMK9VBqy~0lea3mX$a z&v}o^xmdv_1r|{Og&OWM){cQn*DpUVJbR{aykJ)QoIrhu9<@iOv)Q3#FFmc`b!QVI zVdW;XR!I6Lc4mLhv-`iK8lJiGxH7ZOW+4&?H9xp5k{Jie42xnv@D{q!3RC| zZ^0*SH#o`cHglm6vV&uToRfqoI)*^nOan*>ul&slcEijWVG_6H;^yldmw2Q$7;vq?LP<1=;vswy%{Igh|C zD3VA-jq0({O7__Q0!@wHH}8aER<}c@JWk3pUh;8Ok4M5{mf((!(McAi>PAxuPQBs1 zF=>d_YxtnQYP+b(j7JRJQ@03QD2zr^0U64!H5~WgftHF#lS{eSK%?d5QlHkO$oy^k zr5#+a2bM9<36-EU0d=RQo8q{L}g?dASLx3)$CQ7G@?B?kt=DB3A}1 zEa6vH#O~e83YM)HFRpxDo6Nk*oz(bUR>NH-ZpRu8iVY8q7#f9Cw@#+x)Ws9_P ziPUjcyJ1eZUD( zEM$ByVIbe1<@)@sV)vNL3!*pu@2>=%kKZc#`nCW3)Bpa3Tj76C`1pTJ8ajpM8%r+1 zPU^qkEBRf|;dJEB^Wai9Xx+Ryfm`zr!Dd!tl)c<3`&;Q-eKh3!{ zD8erH?YVn}x9(Ys>E!;$r#8DJ*P~OSn(fb$e0a`nDdLH}`9(T*XS+TxBB~r~xf7K@ zkun~nbTq}^(22ZrCDqN&&T}d}t5c+X=h^%&*X7E&wl&gAGgpWh=qLIkTPh$lnmwsF zrI(-dLN)0@;A?DixVEK3t<0_)iv!7AW1>XI`CSDDM{5{fxEH|1@$4^ctkwDF8%%I- z#&Xt@{O(NaU|Xtd6_5IVDe+kJmcdZH)J1ciaWUlovJv$F+8*s@(90VN5Qs@K<>66$ z*?&-1w|%!EFcA`1%;vUWNkrbz3E6`coJ9f^=^WF-(2tHm!SNaHDZt!kKytlCs3K%g zM86O4ZZT0&`#;`Q9pl!_`5FVk{$@4B*uE{lhZUEZ{6yX{97(AD*uq{rHu&Y{0R3G} zo|*A)t$qPxwpS+e3|zfx3aa(H?4K~EtX)xB@ zESnz4Ry6K{IPcivt}HQnj@rJC_Q;(tPNZHq{QbZkv>-}uj=`m&CqBn0ZWCpdGc2z; z-X%Pi1aV&cwz2}J3hJbxBkLlEPwy(c9;J0nDdH;A(YsbPuj=vhoHK=QPq0cV%jRse zSK6z5C-1Zq5(5 zhP`^GDpnLhY-_l9&4;7!+@300_Km`R(P5^Bb}Y_WQV|k?7e8;TtLsXOvee`5Gb%#yqC@7J6g=jR9%u_GCyWVHSw}8<;{|d5G+fA4Visemr1W_CyjaLB1?I05& z(^z34L82`7Z@{P+Y>1tf0x?q$jNKVAl>L9C-T(f@O)>c*k8w{U%VfRfp>ON;M=ws= zzjWAZo^YsK;%&HkdwAz%Em2%nN>pB|CClpWhA^vKyHZa&T%W2_E;qhB4`Yu~3G3)= z{uDB=W_tbJe9}x%kvyAq%d?A@?_~Pc8)-GfnTQ8hWYKWjXN7FD)m`;;K(5FBNuZrr z$vLsQAW@r&O`%UKzio(h;EZE^Nc+8eFhP0N$W6$j>wS4e>iAFz%gJ`#kYc6!C`0cf zSVSu>sG0y5A(SD}PTl~QA3*f7VrXP~q!(KtB8abI*Ib)`*{$BWYq>>TC z4~p+oQ>rLc)>xH{jq7Q@MX7!jaFuWXnSLj7>&&n8n(>1-lMH?LQ}W*SG6d~Z z4!bC++eREZb-_rQD)3HjMs$vF=Ox~v(2Aovmp0n?+xjYqnbEFP<8sL@I9)=lL^IA2dNY7%{E6L z&cvnh3t~BADtdIf-_?Ed87VfobINQZ<=6P0SpKQ+ zG|E4}?Ro&|^f!|ox21<(8A-@!=0%t-%xE6M3OM)T5tlA~ywZbwTI{Rf#8a>$(Pkx9OQ8%TpoL<)^pFnO-n>xN_7 zKew2%QU*yn{)5dYA4HydVS2F81@iK;a=T62Ejm*#w3*pwhMqYb`F!#}lmlgwoJNP* z=X**$w3=DnkKUTT|NckA2K~<3wF_@u1(tMbgm{g%-k^V#v~+nt?u+uVoaCkrbcF>_ z4m5AP)H}X&lumbU2kxh%=-Mdx!pn8mCimu;*)q2?btqy9>I?m>lohx}; z=2{%6&CxSPtC14Pk-`?Fv5QjzKF=%^4C5_t~01x}WadwdVe z0(VBZ$Xp?X`=9sGO$WN>UAkgVT!VHkIG;Lfr@C=3HanoC4L@SuG>PK$qjqT#4_PSP zf3c)tXUf{X>9^f-uOdL$HZsi&_pxYP;wg?PBYxldBd^l@0A61TaS0;T!q zebL^`)H^a%q8>+~+rxt?;in~H>BwE`|L()v9Ys;;9{v`5V4SYZc4Q0F<;0pKQ;fRC zZj?k+Cj0ph7tA|{*TzsbcTrrIT$ocI=F^^GPL;QfxI5e2WuK#d??!Xegcf%6;tVt8 zH@e?LCS05*&lEJR=rDL6_>s2~$`s23e|e6Cgj!40l^s#Ket%x5KU)~384l-d*>ii@mlW5jgXmr1YnfF`OLQ^q zz2XmfB#YX2+KFxDncj;V*-;tp8CIFR;v7d~DDHvtMBz5ji}SSxqpisw9#-&HmuhyO zJ{;sxtv6DxWSKLiEaI-$VJ7HrY`w||W)S^>J}hQ@LJ_aT|F} z8BGV}-!MgM#6A6y{?b9?8{0P~*GtSA^LYlzCuBuE57>pYvtQ%m_Plrc&hY$X7XOe0 zr6wfAUsxC>DG5&&g>6xp#sgVZ&yEQOGceed{j{)6OYNDdHz@JAXDubT(`m>0fp@%B zQEAo{=>ubXWM=2*fnL?1yAjy#O`^19mKXftNjsNgZwUmHNR@CHkdZHdE-?OpnF{n# zBP5(uVrDI5Y!(Fv1qOc-c{T_&DBn713d;i`B5D92{KAwtDuf60L7>D!w;D=(_;;%s zKCJ)i-X_^KfB*6zr!qOcqUrk+%fSj^^I_$_q0BoTDF?4;1Qiz7JXy}u@`|2M9354n zoURXjt^VCl@SV#T>+O^Vb74zs@xN%|qE_tDoD*1HPWWv7i(5f%RA0ha`s&Z>$)cNm z!?_hBOwx^-{Xf;B_GZ@>CC@b6SeWj;z?vCWq$#hcZ*Kb08speLmU@;oUiz(SWqA3k zP*tOiXJhkfuO>}F@z)226u)Oks&5S)_^7J6=RvON^E;tCb#J~tA~+x4^dwG95N|=( zE}?AP_9@*Bou?8`2_t=7TN^dhM;v+WbVe+VjxB_9o9HHVgk5QL*J&Wjsf zB9m2aGfeV9KKehuZ#|lLFlq2c*AS_R(IkF=M+Pqo>)A4d$XE=Wcx7Ph)xfl(*;G^@mTe8AS5t z-Hzf=%8`8XOkdYf=CDf}NBvXHQg3;lU6*Ta_tn=LZ%)*g5)p2*G5MJ?qpo2ZWb>MV zHJZBbgc2onYbIUqk9T$Q8`BI7;u%?^Pi|*7DQp((H)da}7RhqK@hJHz;|{|XRG|{!^L+q9?CoS;9O$cKYS0dH9vg*9D?(M7cyL! zgisRgx3C1<6u)zR$-B;pUS(HJow``ObMt2C2Wyu=3jvsG%y`ergE>91I4eq(~40)#jtA&AQ z)y8`>lLHFRruR(7d{^)rLsPwp-V3OfsoqofYz24a;C!&3KAV?ajZj0A*8p zv_FZJ2JS61>IsgL5MKfGB+7o$$xDyS-w`x9z7Me_R?~JS+GE5+wbUca`N6<0;jQfON}Kdj(7 zT%CuN+Tg@nm8XzPFSr_yMXQhmI7BHx?c%`7t7jQ+CqisMjS=+(z*46of55K-B$4;s z-G{DSyT-2ivI_k=3bIE@+aIPr5ViT3ZJ~+KPKjGar>(6G1&gn+$AcL67XAqAj-aK2 z_w(G_=WzkQT*NTmj0b=K5YjvNBhf{)#*mf*?xyt*x!yg1a3~hDA;QA&C2cIxD{oBo zb~H9O2ZGGI#xYXX9a0CdW!6VQL*OPDY(Ni;Xq*$#Nm*tJ!eWGX20SPU)em!*uIIx( z`|_M;m{BSx0<}*N`_KS`=G{cQAZ9@w=H%B+$9mq zgo41UT%nvZ`~!#PMI>uhwzdYK7z_^y1-CBFO(tTxL2e{qe4s1U7juj+HC#H|VvT}* z9hqUETYseUTF~Ml|MBnCp6_oZm=OUCIISLljyT!Hm?gvBg$~kUk7!Rc&{d8%c!oDb zc~pvk9sp3T0uFPSDG&j+V!-PnXHx$X^|PtMZ6&f}7*1BD$NKAxjA#tnAYAUgy6i}pn=kn`E=Ti_T1_lSo-`DS)7_>XK zo`^Rh?%mJI;zN;q;;C7Cu;zuLmB+%dU!@`xevNH=>=K(HCcPqqARcoWh(e00r0bKj zxW=wIvs66rDFV}j#oD>^=Wjns=A|V53`7VL6VUgt%e$qWK7uwk$(#t&9HGN@uxdB# z;;Nx@(=_(mU&tQ&=9gjH<+^KYPr>aLs-#&!OrwnmJ7^a0S@4zt1UZ=aV#)`OVFXkg zZzF^dx&U@c$e0@tHCYhJ!ETiRwjme7b*qcLtKUAw6BT*}Na`TMOoNRx1BMqtW7`QI zLih25DZlaNYk;8Rlm((IUDPUNg^aCW$jWWe!B3b9_|*qbTIohkOOjm@d~oLg=nVwC zMaHW{@&u2Bu5n?!kT_PY)SE3#P_E( zg{%;?sa3MuSmBH`N9Vt>m4^V_04X6s2D4paBMVd3>eTdiH6W&W`})?abUPN$jPy=v zXJ0=^G70=*6OwR1EUT8PYvG4=$a(0YLo(nw-cOz!2y50>RzydD3DLYXcyFsW?>sp)ih<^b?wSn0eM8 zq9!Z>7#+j6W#ab8C#)huNBS25yS+zJ9mb|0J061%y*?1C5IW$Iq0Tci*?+I}WmJ^W zLc}D8>dR<2*@D(da{Ls4z403}uLnBIwZJO@Xq1Z)!uDUXOm+fNV}uE;27L@1$Z26# z+fN8Im>H6NTcjQJL zdRVXfhp)1+C9@wpCUQ4FIuoW4`=;&AVc&TKih_!i_V`=UYqO%GD6(dAA8xEkLFa0ZoIHVzJ| z(E6=uYEo$b@+Il#&!5mv@9&R6zsADvL^M&Wz>nhG{2r{Ptsu>&f?iI1^Jvh+hYxM6 ztreh^S<~3ar86V4dFstE<@fFUkwi^iQ+9H&p&sb{1Y5T zY-iy6B2@&H2zIFW3RrX~x4zO;Bd<|@$9)wVE<@nk7rg>+0JivvH}NMdI^u%z1EZh| zQ1|-fDVxCuYg!=ZQ#S^7n+ifguwr{oQ517~D`+d$TP`EFi*)WBPg7WN6U|ID_~lZx zMj6=SU`6R4U3p5B=4eRci{am~6!utibp9TfMjDF;ilBmQWMmW%GcIM!LRztm)1VKk zsJo!~5xXUU;x~X2c$m{ae_n=0l3@DW?5rw&yOK1U{p`3#yjd6x+?%g}nX*RN)HF%h z>g%mFEx>Zr6^{W?zD8L~D|{jR6Tcj#IRKdW0yyZYqH*>GllzHBAt)$l4j`*JDv2$Z zG5RB)i$8Ntxjq>hb|H51H8nNTF#TnX56ZXdyR)VRLzb7@Hh8jYX1_H57=4?(xbTcM z*!E`q`}b#YOHb6gXNpMbpzt8qh5LZ@iABS91`stt`5&?j(?>WtWhqnenmK-G-57x zer<}OZQ{tss;3L!Iy$I+Lu~86w$wI;3Fz!aBT@z%P2}J7+SNZrd|Mt*DQkktd;5_Y zJ94V?ICRlgYRFuM>Iz{H;`E5YQVAEk`B{`#7rbsV`A5jZK`e;?07WY!kgEZzdj^8; zZWv4^j{OJXslpN{qs^{_gzlb*Oyg7X-osQ^y!N7u%v#=rb}0TVo<g(yN#~>>p&U9i5&97 zHeX^PXZ-uKx+WStBZ^@I0~3>{LyMjbp0aZy9xFdU@ft={C)j+Fro%6sAcqlSY?dz# z-}!m6_C=lRokxW=b&22{0oDHj0g#OTK&>H3DKfCdNbVK41`2-jmInwxJV<8%mDN3E zjQV$=+p!lIsLfT+8mw`Il|d_fG+9v^Ta_5AeC{N z5OGvd0nV-kP1tNQKu!c~GVnZowbM_?Cq|9B8+ly9nww~os7mfy<8ytP!CvBhpzzw6 zQ_-fM5B~LALhokFQjRD*ri3GAOSWUxStnuD6OjNE=QxqPzzz<@y0-fH^(;cBP2(&U z0!dhq(h%P$lr%Te-r%(wyZ}gY0-OiPZ2?IHWLmz&f+hsJj*QR{F+7pP z0j03hZSM>04YK#jSwU+# z_-9TiVg~1Av886lUxSXP&Q`p!37r(%3i)rKWQXmAhCJWC6?n{3qRK7aWL;d1P5n?i|&xv8o&KcGkT(gvP3 zPc>E-54a)EL=j&Jv?r+<2$59?rd$%SuRmIT=>1_Gu9Me<%bT%_L-Ifo4aGv`m+Aq4 zX)1wYO;R3YVFZi^f7+!^T(50k>3RZ;0J)!}mMFWx(u*MEG1R=TkyfHVAV*L~85z2W zKX5~uLd)>%lO!dvkosA%87Oh9ZY{q*SxabTms!1F%T2E*~ZK7l2R z&{2%F4`j7)C38)rSxAS|@e@0!Sf_y9}D=RDW?OPR01(eaY&2nCb`dEy% zQ{Q@{Yu8f2N{vpq6p2sdxyk@_VO zRrZnDFno%6@4F)B&hQ3>BJK23f4(uQpwFrjFTGCy0thz~SQ_O-vN8rMdG zy`AnpjJg+b>X;!m6YViW z$78|HazWjbnH)KhcmJIK{^$;iy+l(wU(gd2z(WG7e2J=`A4vp0*^OT0649>&QZ$ou z>$Yu$lR?^Ooo$gUHy0Qjm0=K1lL`$Gc_M$olkR?EAYI#?xi=#V=Q!#o`YAZpLUw)kIJN^C$3z72C4TICWqH!?KWsOmHOTJ-_H^cz^dI)^`I;9lJYqhJ;}5Ga|!s_sRxGQHv~T)@)^$A zYArjVweWg{`9y-n&dP?GJ9D%l)z$?ZQ)a((te?DHzDDuU`ZL43Z_3^y(jvnyI#);N zjSa8qtf@UYl-vE`@k6J5S$)O!MK*`BtE{lDNh=?0ayQ@=6d{)9dU3wc5E~nG$|nl^?{8;0dUPF{-0n+L zl9(ExF4#%|*YpoHHSE2@)_;41;uE4);uEhqr0RP1oi)79PT%mkus4H~E5wnZURS@V zr}l27U%9Db;pGM|ci-B6eUs|pSe_-9Q}i-heYZB{y#8qR%Th7K_kOI_bjih`@QTl)sWhAG>+QIq?+eX z3)lIXk?t?<)6A(m#CHcjzUOr*cGQ;XTU#zCMKN9R`{~}aC(q2IWvw-PUg_NXzWV57 zx82cu!5hU zJ1lJ&mDtOrVim6gUZaKvx9jhNiocoj8^r3u=~Xg)J@kEUOFieZ={MiwC%C^*y(f4@ zu0}t(dEQu4z2bq^O&0v5Y-JezxEO7vZ=Btdj1X^(%KGt_nX{{_?mUrF3Nd`Zk9@sQ$>?ji-~DeThaCQKJR^n37jCj51f6cJDe5S>_?ATQC@J{TKCh7(Xw zB(jg0cUM{azI1dv2@CUiuM7IyK}1JD*|9wIVF|hZsnp;0^cts<2VEbi)$ZF-|2nT@a6{P_0!gYI)1$+6^fM)RQIXd zbNf8xVROwo7M{{`iQl8gx7IpYcA4qX?fRP6ePv;?KfX66#5RTJhsfH51z2W#@l6Sw z7TlI3)qHD@Yk?Tut*eO!8gC|fW3Mb`CU|WBuEJjNQ^h1V*u0LXxx(V4|HBBT)YD1t zb1c_od@m8$ahJugeORwdwdC2`5vQ~_jAMV=246ibI=Rb%W&G>uw5hSYlkeOZZ|!qS z2~0S_V=K8%cssLW+M`#&eo;L;<8sT2nkS6^^hUK=z0NYUD5j|!Rq#3=;El)W*lXdh zm0wPN&r3&vo;`BZ_Yv-eyfP>j9IDMx5u}m-)Z(L!6=N zHRGIr#yV+X&E}LVQk-@)T#NbBa%vtgIRzXd3#&zb7h2oCDebvZd5ihf(!PKy-=ks_ zLoyw^)ddY&Qlsv#-Pbk#bEb-V^h$f#?J;k&xp6Lbh6r4o>&L2%*sh2Ic5Y39k7Ex; z+^3AE`?hq4h0)_w+2}Q?ZRR0OY(st5%lfrdHnR?$Q49;9D%y>WuOIkZUE^EEn~z`Y zj)pb2Ju%ZM^JKDATdzA({rpYpgr5i7MoxR0^w#N;_68rUB9rGVO*x;0tOdO1XoB8< zmX-a=Igg+nsV|m&`fvf!K`*&h<%K9y#DZSO~3Mw14EyjPHGyM z#1C|DF#2LW%{;mx*dW)@HY59M%UeT-DDU;U)txG4gE~v^I$jhFFUxOy6ZBO$>L(_2KM!tNp!LL`3 zdRH9#6lQn>6;F0{Z%&mHa9o+I6zEWOtgIGfl-(LpF>JuIWV5tEbw5q8C>=NTK54d> z#{?1_rcBBw)eaU;^Jl9H=NBu7wV6Bb^3e3ZZ~OU|bH4gsTMjW9*VPCX)%{mOPA+tf zL}Uti{TPc}ZmEAQ~-eHiKT}24W3djZhiY;_pXDcqD7~)e(cyWe%qd*3RvxSgB0;ZktmM$Upa)oCBA0sN^oAB4*FD^r_4aMK zF1Ic>bzMY;qho#tLR5&u$h`5puN^%-E3faaB|J-EH0VHug4Z%5T-%_imkA_0blZb; zq);UMPm0lw)!$a%^SogxbZ_wiU zxijU5UVb#o$h7<`d$sWXZ&mY(O`WpMj(3%M)2T(|yQ4LN_lT->1X!@#$QWypGcI1B zV~XGvW)x{WwT-NAyIAkdhdA`O*;Wdlm3;X6fwe>f%{xFyM;@g>5GxG3+qUqK8h}F-(kNi;;=5r9| zX5oL**r8U}I>}e0S{vB#R&+FPo)r)fX_JZb>`?k7qb&&uMHIGw8PE*Ytv&^e5{-TI zjxDrDpc*B=@w&SgG@Hv&_g7$CBahzK%F4<&J7G|^Hf`M+`1f?~!J5YdEj4dz3wGIZ zMCe@VDvA3#_i*Z^=q|6oS4?MBq%)j)jcyzr&5O99|G8mPB*%}7$psRl!xe})Hye!# z!KZq53?U@+5RFty&xcpaj01v^CR07N6xUDDl$Ag0+-9M0o1W_?37lpd_DUshczMkJ zWcw9;#f=4|)PQ}__Ab|RkB$EL*o9f|5 z{l-rspR(r~d&jZt$vrNy{qw;`XTyC3?stwmpV&WHAY1p!ZX0v? zCpR_fshNX8iima>Pdv+|dhXKrU}P&#j*!d3eu}Z3S4}GXtgyeee!R?RF*>9du8IAb zHXBl?S-g$UQR10M>;(igjV378z^a-kzM|?ffN%ktFTbIzk9C2HEy-VN?#tC$i}6IXzX)*sRB_+{2GX6Aag1aXJmee`+~C zQW~6VI%BDw{4LDj%i1t>XeZS|i%0exi*$^wIL&ySvH~6^ZJK4Kr)Ml(rN1bq_eVGm zuH>X0kbZybs}i6{KWD6T^%HBd?@z8INnygJ%>sY`b@?V`k+l%F5_L~Dt617s=8zzIZy z_a6#u$x0h<(CbIC-leF` z{P8cgZTS^_kUvPfDZBQMYrFh1pUE^69m8GL>++fr3YUMM(UvNjGW>{A_tWT`jh1Y8 zcR6SLSG8L=9jX1Fuq#Ttf&S1P^=yG~a`$=pC7nqr)uf8+iFe8I_hH!7$KT2R$ z_H8u}ju?pVkq%u=586C$QX}iLBDYY;-CLq|uzN??rx%oAYl0j`R&nlRI$im`EqSB3 z`^S9_>j1bC(Xo6OSN}4$>*A-=AwO5=M(nRNoP07g)6r${NZTV|%zJbpJ8Es8w_W7T zt|hk;|B>8~$E!-RlcI+z!^PC5#ZW9)rIH39dRM6SXVgn|Ai#(LP5ty{v`Nv1(t-(S z@9Yfp4x3(T@XgF2!>}HbUU`~D4EfM2uAZ|xjE+VsHRjcNlIKOsF^_j1@I7G)AUt}n zCXhHAwudee(MyF2>J;d+S(wO_J~U;hWs4GjPmu5!nZ{s-OjiESJKg(VHg7U$iqK z0I7Q3HWXjDb?a6lXw8S$KngzZMR=M-c!6|cK~%4=ZG`YIm>6n|I-!}G01|-W zQz|`dC{i6^9(}+;InrESE@EmO7F@8>Lh#dBWrsc(%Z*0fJcFAYk8auA_7Ss}Ck(}b za=4fuY(ChSA+G&uuylCfgmbFjh(Wu2@TVP*EOjWWoKnVI^o9A(bMd*g88JH~{ZyX~ zeWvfk+1Dd2uzw-(eqP4)=IN zERB;QBF%uLmioEecDa1Y`;PU@upH?F;WhH~filo1D#kZ&LUzOgh>3geb#*b0#Pxfl z@3(u`t}|FTTSdP2Q=%+Hn~W%OA^GL4&4-SgAI9y|AfU$t>3_-4BTXx;ynTw-qg-8L zcL$z*!S5xd#2SB@*-PpvPmFZ46MMR;0Pn;JQ`2v<2PZGIY@-ogKD|owP&Gvo-v~n- zdCWG1tozsD3&dh%1d-3d{6aqJZR838#pQ<53Xw%XFN0hm>#0;E>PNMmKgqQaTaUMlUrG({N>K8fx6!WNEhYnr+(&Y4A zOwPU{P<7{Exo1s!S9YS!mJ#@gHcsy{GB;;O-p+&cXVz`t7yoBk(rsgX;t=UK;LNqL zL)F&c+{KfYEwd#x=LV!*X(rQ8p0}E0A5Qjux=ddloXg+ya-jN~{efh~j@zfp_8jae z#55`*>1(!v@u#)ss!UwtcGJRSa`b1Oan%W>(H9`O<9%F)h}ho4nsbh+8`>qHtH zyMc_o=6dmD^{7O}jco>J7%jj5$!kVx8%!QpI8~CRyYFqEgSewnoA+3B; zUaZZ?D;TU?eAh=A0Lc>DQ5(StaW? z?(fzWX|t(NNUC5CKxAzI5g@)4kntmyM=xK#eDLVeAmp?RF}cAlIRHZkq#y_p##o(1 zwha;*VXP6s9L^Z*=9r~*a z&_hA`Syoo&NeoHQK_wWTX&~uH6htvbM2Lv3GIlUP*Fey{NA?=Ubd^!!(O*zkAfqsF zTFH@#fxCL46AwUX(GZi{c1Sn~Mzf#}Sj)Tk9N=oec#9MnZ~_F3h^VXq;7w#zz?vtx zDS(2kkd0k$?g9N32rfuL5l|%ls*SD^n|T`59-u5nXmEeS5HG!GmWG1xkFdX?d_Djh zat!O75Fvrc8A7=3LoNV|dPT5YAR%+RxY#4)7p<=Zh8^Jl7UWqEY=EOBS?y@~4NV?? z>|~7nn+2;-t*ciRqyunZ0zF8wdPzxJ2)#Oti(hq&8PiQJ=d&w|cr+>)ww5{e$(9Lmhc z#hcgyxk?XvQ;ClU`RybQ2^ZZ<+qC@`SjU1eku(jvmtcnCg7erh40B6#xe)`PyMB^{ z?8%-(a?Pe_bC5`c`69HaMCFGPe0{g)SoT(+U^FsXUxIr?R4)MDTmOX1zY`Ur_rr$+ zpof-VqK1a&G$zZK6;7482LYH zxSe#@P9nYd;#dlk`tP0@*N-0`nsqV?xeJ6otr41;i`P6OVUq5(!%zFtny;uix*}e& z4e19mE%Cgn?bEe*KVng<)Yj%@p!Q_mVT9?=iRs}=$JEP-g+s7bTRXa>xQOst z+pgnMvtr0~+!wNR;ufuxOLpXgp@mO73NgmtKH3( zGN!w`jIk65fRyZA+~e2R+Ky9Td?SOIY?kcid=S_H;M;5{`(gkxAlOEmwrw-S5ktkL z9ggFa2QeJk#{%r+GP^AyWd2;xA_lSkOofOEpNI({)ir2hKZB+*c1RzPK*Uh^A6M(O z-eYw+XQ3c5kp&Bn2>vGW0e0MVZcY?NfRbb6#Bf;x}8A|lZe z;CA_C3TyF3oN)7q>O{p2=w4z}OhN~7WX2}ee-NZo41jeZ%t9EYu$sP$gB=Jh?*ua_X=I@hi#7G^w8x6r0_~nX zr3hxv(60riC=N3MS2%UpoYxfPmut4NC(~>W0vbfjoPx`5A266ME)^WdC*0M`uw>B@ z+^1@{RqoQl(Jg+(pvm3eA>bh|D?13d!@70r5PAfVY+)M5lgufo5E>zp5)%4mW>UaOk~|I=bOKbId$Uu#huT+{``O3DgCECyzcu5Z?cbUo#U;|N zEZh)}nx%4Eky83|dbGtPLCI1)${4#*kAMGcKwcRv-!WY0g|M zaz3zo_iebQhb44pS!mZ1rEC)4dEqUGI>%2t-2@qmzr}D=sTFWc7$(+{o&sc_)V6}niV~q6X2JvoYp>FY4 znU}QWs&Yr$63jRIh^Sj17xqe>8m;@LN`uZ{yw-K6EI$?*_af<>jtPqJ+%Q9Tk5^J` zvRgo9>G!;Ma9>UOfjyw4$X`BFq4c=5=#UxB@KTLaWF+L46Rfu1Qc7|+$ z^IuE*xt8~?hGTb2Hs9)3{%x}^`g}C)*p5o8K}8A4pWUMhB_I6u{j|$w?$$jrkxi*~ zlcWF1q}%neb9J80Jx8NYPF&&*JHIKw1J(l4&^N|DS|5#NK#(_B>}$#BAHYs2B;qHL zJ|R*PH{0R42Izh626&u!V1W<|8DpO8>}-_&Q}B`CHff++0Tg!4K}K1=UY%>aq1>Ue z7#`n8-ba~%P8uSAJ)^`mSOfzROh!V0uM7g@xb`c?OEJK%!_i>P!O3zrXywsA`;3s# z1O+o3B9x?$4N>1g{9I{sGoOr0kRpLY01O=v5+Z?34?w~IzWCf-094yICKLe@Htg0_ z-_#TUB*NE;i5v6`3C*L#9R%PMfH`;EBJ=@3sPru?Vf4b_SZ@;o;#BJFFAa z!`b}|Q*k0QjyyjjD{BDQt+<4Q{%_yrOP2N=5y40w;>n(%)DYM-qQ-duw_t&24AFCP ziUVl_*w&nE@$+f-1!dgIQoql9kbw0@08m72jeb}?03fdcX>6pX)V+31Mp{~$p+a%+ zb8(_lL(t>LXOJf%n&|`g3rQm$talt?qd0yjE2l^GTAp*WGD^gAv+b3LIH$o%wI$H_ zoSCT5jN!1?hOj>-RQqYd6kwY3g74d97e=ZVBmHO>X=={2n?yCiHGXAf(UN$uz%MuN zWv!g^Qz6d}-GC0l5|Ii2lOHl3F4)Dl9;Wm#sYuf)Zh){8;6FrCaCoC|lsJkAI#H*` z2*ZVNLQ9a0z)g zQZ?k%JS>*0|H*WUw?7gp- zTo`WC+lyx1otkX8wA19;j;s0QAf{bxrA24IB0xrJ+ZOiPcQc!{d}Ic6#g7~oV4Ep< zQ_&?oGWgc;oP1^I(@{BQ{TuNE6W1e{#AvLTzY9iLY-41rznHhj;b`19HqE->NGsdG z**!ZH*Mrj1hVz1u6HpeOsjw>~zB8xXXgoYTP}dUuOX%DgA;cp?3bp)d8fJXsfr*11 zy3t`}$APsCncKB~-MVLqgLoe?j6zN(B17#cUle$-0_|ug$a!NgGN9}zUitG7Dw6?L^>g|wyXRQ_Yf@c+XzgcLqFR*0mrzkjjGFZmnI zk==45IEr|Y`JmYOfO#&-hYq{_+6})pFbquE1!wLs!KIr((5Lz@V4-^=`;+dzDR7;e znuvXE+(Ab~#J23??d4jH(-L|ZQ&W0mK;|H>i=7edffQP~}QJY$l)p9tfWqYwW!Sif4AF8i=P*XoY?*t|9Qgp>ep)*Eq z2b^e0*8yH z4@jkKoMum)PDR6mto;PKN8a9j(A%rE=AS4QJNEltSNVZ+RBY8|63F;W6eV?}^SVaQ zL@au!wOb8t@bs+tdi|Od4j4?mBi%;!&cBfpnsIQf?>LVkK))p!HDoN0h~+AHk5Xmj z@LC#JyeONATK?q{PlbkdY@3duOPe&6V@6C|pa4sHcf+2_Ynw-NJ2YaNqeJ}n-fU#D zV$_fqI-!~3ERr!`$hnzKVoB`KL{wElzk1#KCzk}IKV?lsIJRhe%g5gJ+{YvJhjTP9 z7+goVSl>|?(SJQML|f*WmGWDO?*Qg^=n000UnxC-n> z_dpeM5M3{gPo#Vzcs*=M2z_m!h=Sq7`NU=Sp`0euqgd?3=gq>R9w!7fzBLMxIDi#D zd@yu-RRWl&8F2nY&KK>T5olsmbR!p*53q+#ebU!@UT?p|_ zw3eC*3sEj0g4>RMVuG;5PD5F?umz7PLv|(RffrqM$vq4Mx3>%giq7fn)@|XCmGF`> zsmbL@@6KaOzswqJ&A3>ddn`&M^p&7ov);M$8^6iY2*>N0tqYVYGi4WM^YihknReXy ztQVF)(+v8d9$G+;PDflHgRMuc4kRY=)h-spBK_>8~J$?w6#43H*kg6N<5W2o&46TB0iTN zv$C*12TW$Bo`?J8`>t^>epa=}(O@-ms+ox;(L`A*l+n&#OS<1bL!9?;M`mV~Q{`m6 z;qv+TtD6|;%aQ#<_IMj^MrD0{{gN4YHIQy(Idy8Fdl)H@AxJ6UdGFe_%c?g6Pu3qa zJ8T>%$YTVP1~FuPv;Z=@J&>kY+1U-Ek+&`y&D3ysogjD_a8r}|xxLUiYoW9Nl(Z#e zu*9e2pD10(*~H+*Z+3H*M@$4~m6VjsfKCTeegJe#*7z@wBPR5`%tzt%L2zf$op`j2 zLgh=2Wz<4{IWi(Rg4v+6D4Mv8hVjuON7kT4O6)`dF?-Nt8Yk|(2Zz{%s6Xl-^x8mH zBdtix^8htO&Z6*Aod^6UkUI5%1DuO^H;ar29YoBr&tvKY9ML+bgv9)P{>{9Sxbt;Vq}T)LuCvzq1DJEfp}ZEdATR(tn# z7Z!$foBk@24miT9mU8RYkxedV!+fZApWT)IC+5>bWwtFote>8LI{P){mb4`}#_K-a zY`-4rysEDLfPs?f)5OeB*5e49hETq;$a?4DaD_va9ge8rJDTUZkU6uV@xBULpdi3;Y8O&xwbub)+>Y?vh!0*q9+qp2HZP&JBHm`&jVj`V zFT)`T6UyG@;~k7+O=f)bX%1i0*~e$b9C{OP+&D2YX!{ezA|T9Dezmi#Z)tgtsBM}$ z)9%9VZNk1VqPgP4#^u`e>$+I5tbcxa(t(NkmsgwKBs+|aGEv%X0jWKF;q-@}{UeD@ zV^mj^XA|W;??f5tSt&SW8L{0z4cjcz`FulmZ4(0DpkIgUS)^jZKU28M4;dJO|-DG$s@N&ZBgCh*h(iu4c z))u8~eE?bo6*@AA?H`nA6E3ID`!2fSfLH&iv^PoPl7>$V@?$#U3)O#LTa)gw3}}(s zLT%;oySp3>u=sPrjZ=`vK>Q;U-ZUl~=dpE0!FRiDHfkqKFrr|mokfES_m>zU0y<5V z>)&K*2sRfnYbO~-+Eo!ChHJnM3I%=+EluKa)sn7r_!iXOAw1GY$Do;b=@7iSP-`6L z_D75DCpz(P88$=aKvatW#UdKf$k5k-WR_ zi2=DLw?W0Wk!iF!rZGL@vl+aIR*M8$bUf7CNCwd`Df6S#V|tm5vjeoJUltGpYX z1h6>R(r4ZO8It}Hb{C>ZHLmpYK_{}_{gpwFUq$_)>7DMCOA$(G%Rjy)wDsNDVWpsD zwbZ0{SL$%KK?gc?T7O+j>TL5`tNz`Kj1V+xX63{_VS0d z5^M5yhHypx8w>C^Yg31Eq>PPrs0B{rOn#3=kfNC5olLm4|fSsy0{A7(R7^(jy5 znbd!n28Sv3i^P+I@+3g02z`A z0w{?Q=){UNV_rgHG0`rEFHZ)JDGKs7=&TdWF-chXCXrk>qq%Z(&xgOm%pHfb8ExdX ztM&WdQ6EnnRIIB#vs&?@W$PSsTSDfq%GB{i#_a_u3I#z2JR`T1SIRYh4wPD=3h?W> z(5D}BFk+2o)Q+Koy;9;w!=||NxwGZWr04T7QWfZ|NLvj$ik3;aKDwIuRq!F6t#aR-?7uTV4fzkmazQ6}J@? z-s>2<_0Zv+G1q=XyLRpiaxi=n`l5HKp`y+FWX_M(a*YlAo8#RNTIws>hUU%ij_kEb z+BNQ1pdgv#ADcFwQlWOg|BF2{ECA!Rb5D?quK!vpLVJW;Q{|7VLzsqG%>nflD4Qx} zOIkAv(#oqH_#QHA{ImJRH~THtqx<(y6qkBaSH2cVAD3)&d7t}@iut;C+_?;q?y>!A zbtfm2$yaZ0RMV~X7I*i%$O1jPa|yuan^o`F!Zzb;4%Bohiv0!s(X$u2!r{J&Q)A`w;0 z$K83bTkq*~3J`2vqb`<2vmCi?sBH3hfL?Z%)#|>655|0{lFp}YCiA8w@HlG*+4hzN zjJ`?VU1h?h5+m`9WzxbYZe%Y5=dz;_``z5EKW%q4hlR{EsaP|9^#+HwD#hPw0?#de#lFQ_0?~GXz zO<@7g8XZS5onNm?DKPXj{nr?!B%O-qQW3R!A34T_o|npOKey=@|JNHWQ#H*@^T*gdvE?ZGZZn&Q*gbM_ zIUw4woUj>k+0pWy&pa+RZ}Q$u}*9r zIE!0zr*|Op;2&8j3Loc#8b8lnVE4)m486uuX)fgpb)*DE6K2LYgf6dCP~|xDhKumv@<04@Bwp{c7dZcF#P??CEz|M~&Ix zc%{MF3FquZ5wo>g7pwZD!@LgA?yvkBb2%%H#kXeRWSiyb2-$R&JDX>rDD|;vvbul# z)blH!n(>O~(N5BIG%?me7L!8xwvoyajSJbmdccZL)zJ{>9YuN%8cXPc!QgKk(K zR{Q>Rin*ojZOOIDBsHHwYI;FXX8CD|A`wwBK;cTPy3q6tLw^U1e|3zt;X8fz?m8j` z)oOl>zrLc;;$d5{k^>=WL`#ndVBIFY9@(}!UKWr590_VPl@mowJQ2E5-c=I)RF zZC4cV<`QMj^Mv|Vw&h`alI{jvvV!OBk8RZT{ zlx8j@xp=ZY(PNB)$#k4g7?-2*u@McQ?{-cI#yi5>=5Ji^ydX9!Yce1GDfs`>)96+^Kv(eG#;HPn@p}7 zU!gv$?e|qLy*!b!s_w19>(s(It%KK7D!2j;F7<{yY+)ajicY+pd)jxKpIp|*c^)z2 z)GKEqs53ChtoOUcaWX++mV0>n(*BU_OA2(ijla6&3Np4?Kcb9u`DF9**{hVS9Zxb} z&gYIQq_sBD9+Or?B0wRi!YEjO4lDjhpv^>5Bo}k2*;RpCJ-^RCXw2ZYwA}V^R_BGI z=Pt$ch{`eN$^^}ywvOve?S`oO(krdRHwU~!RUN{mBwD{-HWa?*E>qIn>V+bG zy1eoAQR6Gs*-4!94EJiv#a3*%kMlIwcyPs0?)p9^9lvWQ4*5InDNppWXQn>OxK1f? zQfSIE?+!II7N1f4<#Ob%naJ_LYf8|aHn39o$vdhbpnmh{?V#%%xN#h94Y|K}(-nk! zYbSmko7fbWPixPgI^O8W5-4=wsjpnCy5rH?gPgR#ON(ZLni9;%Pe)0(`1ziZU)w6< zgoI3;Q_z#$ZQp^0}aq4%1-ST|@i25F0&M(ToTpm6~OBI{=<;uRsl<6yZU#hM@(Jyp_=RPQo z0DnhT6&5uOs;jtmZ~JENx4R;KN$&2(XXEt;6tl2d@R9X;`DT81bn5+tUD6ody4NJ)8Z?L**oGU1K} z+iVJmW39ocd_5`TL6)9|n!_i|;vg*nhAtx?-(D3JXRvNJ?cAA&t~dH;yWuYcOUKAe zbm`sSL8N3ZP5smQab$fW#RvY77 z+UAn7PM}PbM^hza#VL60NHelM^QY5;!3j)1ZBJzUHr!Lt7|)w0BXmpi-D;md|EUS( zki%SYQ+KUi9+<>>0rM3l)#J%a;|%j<$Hw|#0c36JXBms z3;Jna-dk3J8cY2Ljm`>PZBH**!wT=7iKGi|j}0ou`ZehKv#CL=USHLJ%Oj1OxBpps z?cBe#t1Ny{i`*6D>Kgb&`Uvk7Gg0kN4`0-mhR41fS>07OnyP^m`560RbE4iRmxy!H z*}kJ9F(;jw9_njLM84Wzl9p9n=l*Vu=B1-HUtNU{JF?&Gyq1_@;1skBfOH| zOHIXJv|+>LUt7XrZa454MEu&{6`2*Oe3^fJeZuX1q_5KYOvQ58?f461%iGQPeD0xB zkON|wvPyv)=?<*y_^(*hpWo(w&_|zQ+ZO7x&1S5}qrY1<*dPd zISSXQ16T8Gygq*XxOjCNX+b>D%rtZ0r(^W|$Uo8;mawKRp-x7fL%ExGloxdO`!a#e-u(gV-~oWhMCo|KdFFS%;kNUlRW$T=)@~PK$_qS zN0i~v&m@@GzuLyiE;Wo587T?8OgV(UQcUt-io?q=U#-?lX~yL-?T5Hjv=_A7zB8+3 z4JV&?n4sS}uC5{?_}291;LgLRWRZ^kYIjoSYaLgT^)iD>;AW(#m~|w!{op zJs+SAFF$L}#ZsvVW8jd0$Vp(~XkeXu8U+Byi^X>|5X!>cxRlo)J^A0Ay1Z)hrF zvcTe76&K!lt$=|M2!fq)+p;2}xZ7Z4Xnvv4XlYSSkPR&b zo)qr4iUA|`B4GK?iOMrfRG{fV)INxVl>PUXKFBT^V`>L=J}j=w7$Ns}38H<{2K`ja z?!liK94{lma++|QS}`G@4ybuSMN$U*SmLsa`yY=C7@!%+w6^;};cYh?&2wTq`1voK zSK;TEJFg_59AO|H9P?MP@BVY|d1&O$o=p^vv`zX#W%v2oYRAKbS!ER?=)=IjiUo3J zoQ?T_BkN|h&#x>_di(hF12;;fb09?FiBkGCwBss%3Pw=!DLjksOLWllBa04!s zh_I!k{Ns!Q8i)c&v}!1@J&ZdwvHp2IlvWDQqBE@l{WHur4FOTP-cxw+#EG26tD?8V zV;9NJW5LdAzCmzQAJ&>JCX1T~{fZeFcz<1JzyI%qdqbyb8=B1;c8{bE{nDY%`^59- zO>;P7_`4^gT@Dp~7x#}BZ7W%|$S-+g5n%V+BS=ptic+_+-SCf_`yH~2sw^A#MCiX8 zu%Y;WsC(~tF7*CyTsdh_8Zx4kO;#wIP9n)jC6ql=vNN+9W)WqtBDAd}BNUQdB+1^2 z%&e^6^L@^_zV~%q_x-Ai(XA-Uvg3}=%-Nv>ZVlA8r$N+3% zo3W5Eo8z6*^3Jrj-JZ8$x5l0qf`7O%@eZ1Ed~Vn&PfICh6S8P6$Tj%UU8z%g$>5yB zwIsoZYf-unBwX?SX!9ilHluqH{-)Cd-UkJu_E3jp4{BHPWMQ%Zt4;CYq5NaRxf6_r ztOY?-@c~t4t=^{dKTj3pc%c=^_=XJ#WGf9G8U&RdYS6!fRwo&|sa_i?pz&+IKxYqF zaO2P`A}avE{6{x-hXp`+f}yW0FH&Yg^a#&^5+u${@EJr24kW1E&vc+5r~N)c&>p249 zeE+kgIMD7v1hm8cZNjuQA|2Lq!LSt~-2+fZl6sJ%!%l%oLVzm$@NkLlh-9c>C3PLb zG;88LgJ0kR$aK&ara*i0o@wl}p{J*Rk7fA?1IW5Jgd^Do?InWI_DM*5UHFZXiR5XL zfcA#e;<-&vx$VdCJerW?0UYS2K}I_Ina?luwSosppry};_6DK)uUqXUlN4ajGkmHL)5Zi!v>;Ksn;CCp+o^uHEMi#<9y#5 zQ5b>a2Eu^>0}E`ALp^%HAz_r*$8-(WTT1K*Jb8c<4_w}~7lWD}`_ORb-9rlBZYQg+ z-_z-=phPk?l+TB7e=Ir9sHGq~c}0ApEZ;0v+y0i6Yf0JduqWvmW>d4VT}+=}e%_tx zF>Jzjx={O_Rz{zz6V>TF{r*0|osYKvsPbct&-rQ!K z&&5fG)mld5xc82yhMw`b_weOu)*o+6uh}8`{MPST7X1;wkE!ay6gv(XmNqCf1o<93 zUPx~?>nQ%Q;M`szS>0=fLaCFQLz`Ugm25q-)QTnV`ZKrX5z?jL2PMoVC>?sc_CMV^ zu(HWm7OeHc6ekk6Nx*2x(;<4=usY|xS_x(65pc{4NUmhUfOgDCQ_5)Rdc^{q^m@af;G36SUs>TH~TbSB$6j+d51zmhOj$H z2qY2h;ywn495oc>2SkG?RS{~W2q6jusmTzLVEyf76_eR4W(9(0qhXP_gCkw?dY5<{ z1jyKT4@f4Qq@y3!)5Pb~8a+RLRFeUSn2$$@J1|han8(=*NeLQ3CZ3>oP|bY$?nE-M z;CYe^ohu;6Hl&m!M+=QS*fJx~Q8+ffxe4BQy;D&WJ00DDA4!9bk7vOTxgHgzo%mBd zQKky=MO|E>o_ybzucXg#tmpzxOM^S1eh#(+7J09De3M**^Xdq$d?j4@mU-7Z(o8Bt zo>)D4;y0N5pqJ4;Rnlwou_m)y%}Kn~GnTuh5?vlCC}!;+9^E7r928u)j=|&8trjR0 z*Ml{%@APr)z^};E;fE2l(l@4DIC&wLxv`M#hHH0x zLHO~rFK$x}u%yH$KGnK4bF4Rh&F7I`QU0RB6&Z!NsI()Qi@Q@h?#BzIQmRt$FCOrV z)S`U6ZE$$Q$SLmVgC*}B#Q3fk^mOJ4RE~)rmpRY=A;w^p%a~=JozpOJj}n8IOj32H zW7X%)b3M~u&7V5vJCB8O2Q_71+#5^vxb-NTUrQU`-tDDoH|6TjTvdDRF||RCnoP zz`%^(A3<1)sj(xn7%2NBNwP&8{E<-3Lx&<^L^*)OTDZ56l*g}=Ng*#Uj}BH@Lt_Z) z`Ug;^K()whx73UIzs48`90JJ}h9waJF~nAWScMF+NgzRiFX2d!#`ThenZZzrz*|T@ z0rjHPh%4Y*h)E!(^aFa2oP)1lb%6OwdK_BKX+epojvEzZ76)>OAV<#5$vF^5r~>+P zg8L{CO#(WI6$Pv*p6NY4tsu<1dzbHp#2=@h-!Pbc_2)Qm9=SL$-HycIquh-jXZbMm zdm?<=-S+EqzC(2(`b{Sf?kueT#GCQp0pF99(+0o$sL^orUy0qaZe7KT?qt4UYbmj( zyRGvydex!k>~L^XSKs3PMy{dG9egkL9!0H%VQrqT|;!`{{zTINm`|~|}lMNwWCrewe z^7{g5>=c7cf&^_p55i!2BVC0)9p7tPEqha84gsO9b zx`SZO6TZps`7UQjb24MAB2->E6X`bl%#fupEo&)#|J5SSYv)pnn3@Y{-gbs~F1K&K zH{~KGjD2$J7oS2>MP>dyvto1i7EVju1Fv~p3p?g{gm&Add@6{JR!C^mmyW(<#o4uT0BBD#qI z=y_7fumWWKN{??4IGuJ&l*b)!HkK>YKq;Swsbp} z4qEi-NEfL#>qCS|g$M)Mfn?YteA;&kjKv2BQ&XVD?nT#Qg3G^tuL|Iw)#Y(-H}pT_ zmfKi2+-$!4+65vVa{g<`Odc&B!LTgvAtX3lQX~^Ed@=%;PRml2)zn9-X zRlIV;o2|XR;SraH{2i-Qe;Ndy>|*x)Bw3iEGk2aPB}w2q)zy~+Yu6-F{g&4fX;_ik z?Ura77kRm?uCiibCCxW;-ecGBS@9cDaW^ER6w{+O{u1ldlw2#jx&FS;VUx6h$V$4f zXoa?26|RppH{!^~teoxBb4MpT)lMIMo4PjgUs`}J)i+-jo?GpbQhaOSa#bMTVAd(8 z&-w9T4feBzN(HTcc~z74#f#P=i7!I>o}=PGW*GCBr-;E2c&UsQQhiZWF(cuwMP2jn-GUWz)Dfc(=(Dl z0o@?>j3ki~!7hOBW`rdI1+g*NCxs`#jaAjxQ+*JvXsH`@mPVpH%5AEYPap8bL~%<3 zeDJ6^IQl({7Xdbt7T5O83_VVL3rIR~;UXWd428yYsyrk&STzgi7D1r`amvPFe?C$% zAnqrw38D@|NXk$)Y=<)J`Q8ZsCNX?Nt5d77(vP|AZR2I!!U1a%Hw4_y;kIw|GHAIa zGv*>Yo;>)5&h^J|w6={?^`#3JV*3j3In@lgEXO4#51-WhxJF9r&vf45pZA|^a-K@eM@&6dL zgNIG8yL;(0ES03rOYGXs#?EdlJstAerJtq4qtT*$x0U#w=guh?xZctq31G|abJ}j~ z#q}g&qP(lD`)cmvJ>`ix#~%*h!V!uczNgn|r7hS%S3!ZVDV7gAZ1ExiMeFnlvva;~ zxleYLrPF|3aJR4z4TNxha zD0}D9eWhrI;5kLpEGs>L&`obN^~P&jMp)+bZ(m@l|8jRDOKAQ58%cq{@niD1H|1z? zmRS#7R~uTeE=d_xdUM1fGU6fsNSB=dk$_b-T(l3MCI0+GD;R>UA(E?a-S&!L9LU90 zfL|PKsb(T?&>EtQ!ktz1;X^m{fJ;l$4eJJ*Q5ld00|_BoLXwUTd*luV2A5+4WCS%) zmco8jX?+MX=XN}8q&bjb&j7F>C4+0$vU?}X+YxZ^s2L=@S7b2mn_EUEp4ekCsT!&| z6!B)3meZ(Pq$uh9NmvQZZp>Es4p+I>5H+!4S}XB-X#P$=kum7?8{ZSNKN&qIf)C zi`+z=3lpF{oxrc}{rOYP!h&=n*dv(Dgh&F;vzJz0?PF)BM6?~6k1F6=O&ap+UJtxF z(*6umEKH=$1P%wC0qj1#c+K3>Oyu2v@qmLX2u&?o9ebEqE-wW1AIEoNhM_zvkI|*k zhT2*u8+KIP$53^r4_s0! zwWQVs1_TgD;IGIRpag=3#j1GgIaTh>ufLQEE59ZsG^Ti4Fc+0HJ_v>da^R)E z_mOhpzMtE!s=dF-D@7*-&WR_MT$Xfi z7E;%9=P^CI8d8zgn5zCkV47*(>3G`Z;Qobm$21S#mP+*;Uj2Mqx*%$qu`HX_l)_GQ z_>oq1+{T3J6g`hs_ngM7Gt)cC9q^@jty|Hr+g4(gx%(?e^`nN1yElMDkR9pKSN?DXs9U3I0W>sri~Ae6PyQ< z!+1a`bkPJ9BDIh!D0IRDLH!K1yk>!u6%1OQZ4SgshEmaQ%Wf4L8$ld_1?F<-Dv5}b zOnB15;JgyA#ISOxfX#5A`UEwLX>%M)p2pAlJb>Ys57BLTUbc1%9R9XbmRwg^`E1ekgg&LgB?l!FE{Qh z*9U(?xs&hAhn#2l4O6u;|B_JldY_Er6f$le{i+E^*NfJSE46K(Ugs+q{N6nEm-f^7 zKeV6oV*}=Mx%Uvkdgfu>>+_8^GMnK0)zsQN@bN*}b~3&xAKvAEBXj;{ik1{xDD!^< z``YUZhpc9BK)~xR#s8E7-f+F}eavOefANy|-+oSf&wqc~we){emEt$;5hf4ezvKl0 z>u>$>zbC-8V(@!_P;%Tx0Zt9-TSTJ^}Yc=QwK0|9Fxz z#MeQqyf^RgNND+V(@i?AE6?Xc#~isWMgz(ryz5Tmr0x*pIvBODMkB}OLm!1xtm6$@ z=_pXCLx$D19~sgaSM?0Be3xA0syXR*u_N?~ zT*Gi`PXVDniBw_=u+r*4eWt9taNhv1YGyk@($)V8@ZzTjaPSVvEflylAAoY^hdn@} zp~U(5(r#3ptCiFL{kH1F)L9%}n;&|QC3ttR{+kgejEb#Kl5W}fzHYCmUDtEim~fCN z0bM3a5rm1FVc-bid!Wa&LqIYvyjr9R{cNu+D@##s(nanNMCfc}S_0yUorXX9A}9_= zrxNa|@#9H8nXUB35A{J_NH@X!3;*Od;MZoKL-&xet()jJ(6m2*zr zqjnH_slHwGd2Rc)s!xxc+HVUwJ!U=<>b`H`)qb9z!GV%RQvSWW-S4pbnCfXRF8r71>do%9s%}-Ao zj9)$ZrDJz*!Se~d&Wd~}i+X)_8OY8gbL*?vQdW#&Ur_DjI z%?S#I-~fa)x6$M5e`)Tv_RenFeK97{)U-Xe%k725fkVXxaceXp`rm9GsuFHP%^GxC zTO_6{ov&v7=cCPHa_v{&=8wM(I3p#yr*>-1!AH7{!Sg5Z$Bkz; z(oOyA)ircF2Qs-t6EkWu1d_E*s zVNbffo4itn)Um&Xm6?+=ayIT~OMXj;dYrXr?s>`*qFR$v*QIuz@_okQb-o9Q&sa~z zF)&AVPgF2lWq_YrT?uC_aX&C`|x3(K8E*RWFZ!76>B) z_wSsEaAqJ4gUkj&T7^?Y-C&SX?4+j&ahs-_%dP z>zZ%R=$lB@ep|}G_G~OVox9Oi>+Y-GVa`>NRC;rE!|Z+Q`^-D9?Pk~CPvx;SDltx| zN>f~L^mu-CVt8vaoYa#5`D@kd)i-8HwCX;n{*bn z6K2aTl?rUgv;(?|VluGmKKtjE2rF4W&0?KSE4x+oZxlm)IS}fk+V%<42yV(%aU0n9 z$jh6}S8lVy-`d9m6W_x}WfE=>?c;df+Jg=22iJuS~na%^u|5-wdqwq)2=9b zRu{q32RGa1Fmt)%_Ih<&y=UH*y_pQ^|Dc?4zDM5PrwF>U5`kOW<3@s?%wWYiw}v@c(qE z*pyfxDcKa1cRcj1Vt!Uq9+WT&l)X&?s-5m(R&8tvBQt_QIb-G5AA_St9g|4XkH8XY zA|4+RErD@yecul3nl{*vMx1Tmz8=;S>y5~;FceR~4o)F8+I@DEC=dwa2d+fX{_B4& z&R$zSu{+GeR(lULgt*QnH8<(emJ&TO191M}>hblk*XobL}e$+f!r zL%Y}Q6>SNrcp>uOv>P;p4+6wia(ZYlM$K}4s8JqGQlc_cs~dQ{X(jsk#Whz<$}hGk zi?>Z2l6+!2*J>uCKTYLDed2sNPsm|WJ6XLrn^%&Jb0T*lO)IRm60U$he%sRF4n~H36<;$Lts%1XDmuyOQOE0FFOnM9f`Bvnr8;^$-*@D&(!0ALQip@_(z0 z#&V9NpM9Z?0mU@bZR(uK7bhH#@^$SAsi=9FuGmZ=C%sk8BrU=^xH?`x3jE;l$vCC& zoq^C-Ev_oB`9H%6-eMG8yEXQ*?TgcVUelOvyphY(J4`Q5C9Ty@Go!5L;jw0;W}&Qy zX6K=8)y;SAJa;7|MUx4ND7EK!Y_ZM3vnC@25mT*fqUDg%2)3D{}-8;r_(=PJMZY_&GIE2jsp*0K| z4%w1j2u$7B3fP2VZ*4<)%+na4IEs)UW_T5cTb@q|K5`~Iztrf}{0Qd2eaDrTu95T} zE_K+t9pSM|bQ@$soC)!Y0h;R{9GpTsmucN9f)&1z$owGEDIRfpk)y-UubX`X&@kmm z7&6gh->S1<$3+SJx~r{v+kdV91v)9VmbQmO;A^7`xq~DN1-GFaUcbAhuP-g#uunL0 zG<2|8NCQ_pSg674A)(6yYsuNVp1C2%)J&GH`~TFg!fis(C5(XXvHAQd*JeXu14q^c zL$T#u{`S8ds2aT8>7dbTR&kv?%ET(jIY9PY%DE`x-?PCS=DL%P;=;dMv+1TqV46{O zIJ;5q+g0C({QC#4wa2tSXY(3!dy196VASVkG5bt)W_#kTOWGmOGJW!VyG@mf!Tk#@ z3?{S(cqi*U6R+0heSB$%dj&7@yLYUDO<;jT4VjIpC;qIQQf+k)+XBgklTYCL&<zto@O0vr|Y_kIf|Ap-Jv~_fvm>P$ zN?p}l{ta|tVh`(hcA*Q~MOUcb>?9R)0%K%fqQg_f;F~G_r^p=4&!3X9{8wwy>s=Sn zXPeAOaeb}QN2gsu)Q_a%MLVn-4;t5X%3x3VT;u3#aF!$Cg9;GDKT^1buT9vN+l zKVK?ZT^VTfi#!$dZRTRlbI*+Dbn+_AoypJ76`VS6`|})U*va#EJmX>b4IV=A~56TU{=6b_6DF%AIrZd<> zm1KLu=(M%S_1MS~6W#sJ91|Vm$7wFvw-`Pd%Gfn~g(Y?E8@Gk=2Q63hhBW3r6&f>b zTKv5_P@vdfKZ|I=Y>dbtyghUxQ!mFLn1rxzWTjJ>28-uCkUMqsJjj_d0jNzTFd^N= zFzXo*5kMWgF!8ah2cYu7hjD+|=6G2D5~ z^**iI6Z#U)k$n0prVyEo(?@MsAygu-J0YaO8{m&9p`RrD0|K7twBrz4BcN0eQ}23W zEEDs|cOr_%%^>}Q;Z9c82D}fIh)8nn_Fez=wLLpTi>Q$}5!^;%Z@^fP$$iKJCS5L4 zfMbD|4Y)K-e{TMLeCmC3eHq>;dkGf8%prG%D8B&3A`hr)JgsKuaM7S_VN>Au%uE{U zrp!!!>YA6rmtQbx`S}iN>HGR#lIbihwI~>KydXhiJuA8@<(NLF+C+XkWFwQw!6F&I zJm8DlgoC8w7^~+Eq)R{+Y)pn9+jRU+mU4I+j~Whk4{OyX4k@W`vU}wr$0P*K{^c+R zmM!b?Tpj|WA_X51H8L8NUdZ59Qj(#UxW-4xYFcbjb!^cnA=h zS1`R~i)6TXZGTkvASli)r@9yI*V_POuru(+tSpWNV&6Zzt504t# z&dqrl?4g?}oY%07EHfMY&3?5+Xk)t7uDGb^ zT?jujad6=Mqm})&Rlm^r%6ME@*jw+FX;!b51xqqa8&-q%%SGp88S0nRl$GxT;6(l# z$*Mwl)O*NIkRHkjFdCkQ=?3o=(PN8na5X@iR0G85(Yahc%v+R(CeQ+C53yT5P*{KQ z%d_JoU0h8q6)xmoqpf!<_jvFhlZ2#@bOIz1my%| z&yYwf;K<6Vsx(YYOfnOtl!W7X-$V}BjH0PckX7FFSZwwZBGa7^7Yp5>1O=7_AK?2p7xv2L zRj-+toCIC9+rO-5rFIfJv+d9%K+upNy*r$YRz8j^jsRqS#)P8!r{#C+4dk9Gbh}os z>2T|!% zb+Y<6C`$s&mEw5nUpUx5pV8xyEPGqpU>e{hamzg-C~LrI({=4VX0!Vv+83>ZHO%|P zCyS=dI3iB0yA%HX;dmoukxmMSLTpz>y?m3aw<5o3($i8OZZR~lPCzWdP&|m0zL@!2RqM?P9k-o z*(DZ$`tlU=97xlLq(jhm#yA!TCkrVS84vD9!*m3~6{Z6we_*3X{y>I>IdHhzMA_Xc!K_V-IU=114ZM%un_mS57ZDqd7P_+%!J)w{q`7y)gU6y zshh-fWGzFAuD$d|!gazZWG)atpGDUF9#j*M);qjm@j(j6Btko|;fE0=07@5(OuNIl z0_`3dWdu()2;2-BxkAE)SkH3~HAz-`{8MHI{e6e!EGqB%-m1QbZzxvl+iMkJLu(B? zZifb`#PGVs{aYHd&Pwc5F=!LvqK(*pB*x@LD@&5Klv2kdlQSEI{y3a-u|oIv$i`O{ z>8Bpw8wtIdq-4&kU(3++QX)RO;xp@t*{om6%ax)0)Ia15 z6)G!V(f{V;a$J5_yEctanwI{|(ypX;UDb5OgYJUn({Jg57-o}=&08AXy@j-IJgCDC zZrb_GTROHr*-(ArY)9i*%fhrT$J%9?3+L)`KPxr|9Qn*|cu%WPY#4i>hG2k9I$_M#!Tm?1j=5C2hFHGd@hZmou1iTks(QuYJtgA9Dis4;S?tERhPY2$+{+pDP-v)o*x@A+v59k8w{T7@^sHUoglwY8(8urm zi7`b@Z<3m_=c6Ju>BE?RA#eF^bhI+e5o_$E5>QlRxr`gQygS6sejOYC6D~?5Gke#W zC#}BiMj1mwT2X1`=e7QHM!z$t-tXt)88r3Qd?F=>4Ui&Q{eAOjv3~8t|O7V zM5_jy2haIe{MehY>PWg+v(24aet6WTv+%PU8A!gxpAtpJ9=(FkMj3Q^QWs)iQlxDzGw%HS+tbrCW8R|&f%#|8 zMtZE~Y!vkNUX43de-r4~0&M3b)e1ZDKCTs@DNxF7g^J#jU1&M2@XQfmytYSfZW{QX z_1E{F*QQ=kH~QMLo|nG+c$@ct z-lB9i-IP{9U>^OBqMMXDma?t0vhmXE!I1k>v z6D)CmyD$Ur*C;Qk@WRkiw#q%8w{S@~ut*~j9ofR0594fg!yWI|bCW>_RzA4Xh`t*B zuLI`QbaMwl7&{tXc9(>>6=oa~rHxV;1KvNMSRfU<&+#H7C^V@cb(R;Ws) z$BP`RrWR*Lj$sdyS@c=0kn(5WK)R-0mKqxoy5v|kSm1OWy9|Xt$rMMB160}&Dmr`5 z7f);&m9B_D>sudDuYd?~RATV(oW^*9(%ho9){NBdMz=Z-^}?Iq(dnjLGU}8N=dm?5 z)_pN&zbmrjYisCtKjX6UQYtLcIP>oBe>92N8A#}a)x4PXApHeGUp^q6f7nY9`!?D4 zS3BR(5j@Zo>^>C#TbY?_AT9~c%M%AbUHeA;^>LCmucyZnX`UaYAnQ7Df9%aUA0P{| z|3VgSx9k3K{wPP(vsRDTiNma8`*$ABq7a%(`7_nz>fEALxCl$_{Pzw1J955vWb(hF z4iD+^m3#KhGw3rvcS(8b#0kA>^N}a#vl#cgB$|zXa^mkEqO2;C8_Hu5HQ&*+J6^eJ z5DQ9rOM14^NZ^Tjz32J|?yN2@@N9hS+?b=r=~Y!z`~ZqC;~MYA zUOuANoR9snk@lDIxMs^_71I{Ej;pnvdNbKKk1OX+*Cn2grne4Kv-l*$&C}mk@T9$_ zZ12aeyV@814ty^xY8liN3*(OU~_xUg*7Ba9H%C^w-9H%p9HFg%FcdOG{>xEpHR8q zB*Z%lGrfU@!TG1d^R1IekR#c6r~@({Cz7TS7FRgT6!DNy(m1fr&6TVs zpoYl7!5FX3jR^ymC~)a`dkdcXL`5+`R!FAxpspc_Q|zHKN6>B7LXXt5l-E*p876-4I5%TF{vC9K~&X5fGfHED_ax+@9+>1tS5=<0}C|^ zlZS{47%$vLT3SnJya~8FhD#*c9%Lza+*b{kW;>$MN4xh`R@$ z^6%pH3PO21)p$Z^;Hyn67dUA#Bu*1$IVxe}*fi;V!DQ&uz-&_AfvSkx*$@e!`D+Z`|T2W>mH`38{QkmT)2nGB`w0V7nm$^ zF!Y@uLD-g6wY6rDGd???7lKfqJ?_K%I^9PE0C$qf03^5-Ido`yd&b8#koUxcY55k~Dww0(ek|B06g1bRiR#3Q;Mr@4s^PCmhE|zS^6<}P zV#|~KN4T~p=H`Yl5n`}wbw!*4hlUvvh)}EFgg5UKKmfw)pdHc8y|@$C06X9_ux~qd z@BV3!ynsU92qObXUJ(F*rl;CML_UEInw$n05(GKYSiy)O7JY+1tCwX)8=E=#W7JS%N5frk6kUBjlHD7#Smd*B_F}>Balo9 zfbYNG2&>(NFeDpzK0Y47nT%eq62AyV1P5*YGC~%t!bovlHNXTWfF!6};9lt1 zZZ8T~A-r8#@M@9S&&cr&p;qQx{r)!qpZ?l@r1Cb_fNIkzAW=3UC9xlN!is>8ur`gH z)}NW@>qW0ms>es}*dc6hAiOsNj1QCL%J{B(E}JW^`}g>c$TJ)Gtv_+z^Bzd0ZB+v3 z$W`q7oV(TgZN9zKMW<4A1(6p7=K5bB{Za{P{qE3Z><0ADk+)mIZf{B4UO=1w`f#Yp z9&n-jU*ACfi-Q|q{2!m%#Q6WU&onIA_yn};g6t3LQrr6X>-g8dc{BOWJfa}1mD@9{ z?uCv&{_QWr*6zo^?T?Ms|9bj=|G6$?d;WiY=GOnc&schiAO4?L*Tz==kPDAZ>RuCRY~t!;RhGLuQ%sE3$M>lMe@(M~YpOoAvg!kcKh#;H<)tTV zx__|88NdGhV{|(5ZHyq@w*1Sy7VV4`;cda>CDS%Ocjr#58amj84!UjI^0%Z}IY8Bj zBV)Rdm%PRomCiUaWp_8LUYd$eg1T9UsrOOD9po?3YVcq!4fcO#}zUz&S` zERsgRhM+G>UQ(%Wf_~jV081>P}>-MMb>POGNy%#sua zz7eN2_%6b5f>=SN=vmcA5>G@MWE21nITYn}A^04aNkNGA*-V@pe~p9jULG-&SscxQ zSCF$Lj)MW@*7+c&JOloP{Yl!0Z&tzYaf9%N5kJ?wG8Dww`GaW}lwJhTDJvx9M`gb) zj;3dkN~rSI9!+;TARoK@+$=GVQ?dzhAz0E+xKyJL=Q|AG6o&hSBo$(8oxu55wbQQv zVGOt>utA5|MCP8|Vipex=X5Nt9Dz$0JrzmjAUgBMr@4dZc#ohK1y|}EVpa`<2``3B z;0SPA-ab_x@!JY&TM3pe#Yr~n=8UWI@*va-)?C8 z?PVaeDUK)_mM$c=-^bJgsB|Y;+Tc7;Ly9|~je}(P4=#Q(;{tcW(CPiQp3dnznv9K^ z09_gD(nneuf726vLPJ^iRI04>;B+n6$ZN*C7`ITyE!UQayFOSPh(}ZeLSQi$Q4Kw5 zEBI0HQ3G(!)MZuI5fX?r1-PYz&VOm{y^5=v308kVGe{M=4os3{x&~JRlcob=SSh9k zc3X{9pyV>e71ofdomc{%3-3+}T=J@kGBFS@ldC*Z$S~wTcuK(xlPbgIqNxf7!wHxS zP@?c72Y?vXu{$214q|8yJ3r9=nkPs5f{_*Z`v|6T?1Z;nmH5V4A?e+s2~p(WK4A`p-vbsG9zA_w(C z)+xox`07e109IkN^T?a=(3Au>wGO*J72AY3F9@WYbRJb3J~se2!T`7wZn@5xId4>9 zq#7h$0V<$q4Cyiu(Y*>`jQiYJ9f=e|CFh2Y4jQ%w!=I(e3qZ43oP8GS@8+a|4!;|jTA9jq$KcT&F7abo4iPy{qgA2B5h6fqKC2H`_B z#R@s1f6#UEn#KUjbmM}Put{fXa70*!yd@u3r zVL1@Ya!w9jL}SeDytV5f;mX6}<7-K>5R87fHOT2sc;wpHgTbgb$p}1PpGxF~{u~h- zOOC$M^)dG*LIDh5xwwr5oRx7QpEy=q5_T5VHBto#Z%w_## zbNwI5togE0!#lR+>k<|fcdmN+U5IQ5(@FG2QYX+kdO<&N1Dc#Q6Cc<^h25si=B(3k zv%qZ1LefZ9X7ftcOaRQ?Mn_i;+?KW!YfTxe8<`eDFI;VR2nh)l6%~=FTlm7daVTGj zn#2LxgwPHQYFK+1J3b&VaE+dJOeC7uXV3VN$8!AS$&Z{-A4eNgsc{qlHFv=4g8Tg) zROK1z%!*?h;Kh91KMQmP7d&~aWW_*+YJqG6h7M_4`~lg;#8=bM0Y%5@x`E%kFs5o~ zXvoMmcs`p$4h)>r8L`flJizYomZU6R^M$4n(;&FWvPU98$9@f-Fpo=F{K4p<$&`g5 z9g$i7^rqw3VekpE5&7HV(@VCaT_I=UJ_cBADX7#KMEkUEQvab05n z(gLJF2z};--hz~Y)!Xo}F#AtaMM2-!1=nkKp053{&W-M`qq4TPmZe17vg?winOVPb z>EPMFPRPGhW19qiD1gb8F9@qxRtAH$?607$)}N@7b2|BsOUUJ&W%bK%R}RX>cpO=) z^JbAFckBw-smOS>rN^qVw-zf1JLUTD$W}mFp18W(wU2jRY0<|aGlj~=;9!^sj-24J zREHJjAyj-@ckZk}bN@ZD#{*9Wg&HdGNDp~J&s->A0c!-nqQ96n3N!#pk^vMe{do7s zjvnxVG6)6TTD=qdU9eLhKI9lHUbG>Ug4@hEs7O*GVZvEDC@pMkW$lf@dj2`0t(b9K z1E(+UhcxK87zFh5z>}hgO3-*->s?%2jEa$Ei@(RozIT7pR6z~&TM(s(H$*~mxk?UY z5JT|jy(Y8Hi-+^UPLWIm+(ftWSirlJfhYK(ykQ=c@vU6yP49Yy56G&i(o#2B7@L?R zGTqJ-}w| z`3i^x#=s3>(`)l5;wsvJrYg<&;IS4xKub{(f=#Mng zePJHC#+vGDLm`vLsrUIc{y_{=Z2JxlS>KkFU;VEB?bV|_)uPP`R@0Nt#mhhGa>oRe z?Dh#IzuRkxnPw_b1ww;ve~ zCiCl~!aW{Mwqw#bS247b4<%V22GNq-P$HnBfWdbzd|(a&E0pk}P=f$|#xFC$eb>@% z$+i}0#HNs)g748pk%lWLwr?c@qm#=kDm0PMcGcQ^1$QYO;c4)h;;U|@xzE0~W^m4t zEAc9}H7u@WZv>EKQ8{JfFha^Zpxe%dTO;AW#Wa)#r2bf8s$gqtD=x*t^=pLUH?p-f zSmg;P(5KZoZLVb=q8bq3=dU?<|0+xmNuKL3F&UeLCPSi%# z>wS36i&-~L%wL2d8?ATG-xSjy);1B256<(ZwZ}8_1q8I;FLzpxH6R%lUN@`2TifPZ zE~{{e?L(C8u~V&%9Jl*v)+a4p9aEX#H4x;$#u=C4!fAAFV%qnpT-()q?}}g0u7lRt zcO~(q_e-(978VZ4Ca4Op2gt^})t1(;1eE-+iMw#Y%CI-a)SBgZH5;d>iw%Vkn}f78 zgmhr_`apUwp$;=ePJ_(Ke4o#y*+yW%B<>6gP;z_b>e7(BBv_Zf8ILKfF%if=gXje| zy^d6Xn((hUNJxOumrmk(vB&Hixz!~{5GlZrNw)xT?ibHmwARPf)!+BnWxCH?#3=;X z-vE+~jL_p@Kax0yI*XK(4~~#AiwFYCMs68?1TIxGoJeC&rLSMVzNUqyXS`%}k3aj+ zxucQh7&Wc}ttwP{EOJ+)K|);0R|^~Yg*G4ypfh%IEeL!FR`y}0l8bx^3PNBmFRW(k ze^vv>gk)C~GJfmwQbf){?jJzqAI8(FHX~UZDd)3H z(B+zgA-X4In26;^$`&8v0CXBG|FLRn{jgb?}v8nI&fFFbaEVy!spcCR^vhskxlELi+ zj##ZH7^8U+y!r5&OnzhIQ$OcG^NKargvQHX>)q9S&{L4b#MrJ`>Oi$p0(GJ^e3 z02U*O(m)m7m1PvUpa0Wkb$PCDWTXb?V`kgWXXkQ*aR2wcD)B6Te)KOMroG_rLy+J( zc;*Ahpu=TFMzPE}4u$r#D@#T>KCPhBSK(n>cPis%Nq&jokD;o*ox$6dFIZ8moPD+7 zHeb~JuHLRJCaMgX?e(FH>Xauzx6__$FY_Pijm{k0`#{mtHA8n@(C>jI$Clxh?az?B zRi11A%SJEpK|_1Z!z=cxdd&VwdAGTmk6bvDDnSvd&Hw(R5bd4#9;4FJE(4mh*hsPU zCE}O=hSyzepJGm!6Qn1OdF3;EH@d0CW!&a zSUZ;JQZS$RA70`+W`g2iB!k9@$_anbi1M7|0iq*BQo((c#ZU?g9b2R+yN`H=Td;~i zUyJ(jD5OwCEdhZlgFqcGTht+F*oaDzz(W@<@SsUaMkh2j)YsRSKO*=>XaxEq=tWcU z?PSpMkEa<^q25dEC2?2UaHCrQBO&TVP?==F7Rigpgn2Jdd|2)z zA{f|ruYob@9U3|Y+E&nmRwMrquvZ6wppj)_Zx3etC|YE>7^AAq8n3^lNu>0pUKg`v zV%9Uq8uP(dNaR7waSS*hQR@;J7kCyYLy0T1Y$tpm?m+NVd)_fJR1Oj(3(zdu&rBh1 z91E5%f%bK779Ix>gYh+c#L;PWUY%B(n*&~Bik<@v7s|SsCE*%htlpwAMcV!%lQid3t{eyJBiG_3AMjn|L5hBn7W@vZtDL7rv?F zXT$vd5B;&_KM92wU9Cr7i~+R&_m5Mh1jhgOkN-b>#&r;UN`HMeDC#AfYyUv<s~Kb{ZS^8$wL42tsSF|b8wHA|9~U4;!y{G z6Q!|}?XPA3{e2UUC=)(FQ)=O}UdJ!6XHDAX7U$B&k4xrEucfN8_4F@DzMT**-N?ul z8CLRh$F?f9f6NjOh@a`7kM=X_O>3$75lLBgkkNM6*^pMPUC~gvymh72E@`DHYxcHG z%A=`wdK&QI@}D3xpxrYduM)43D)shLIlQ`GrAy-Q#9H2psS!y3!i5We`J;|LEVb8B z=hqXxp2>HH}R|594 z{pOqb7JWtHY?^m(ZC_m89`%l>P1~}ZGP@H`dY@ZmzZdjiWy~|uR--%k=fBkiTx_TOrDZzShRIfXXxS9oMvD+6J_|632kSD|h=**R=ON<~7u>=8} z8n)OYzLX5vW&lVk)Z+P^s1$JeAx-UZf3*G;-y$Ydvgl9)Pl1i$cOJ8mQaW`Rf z>HKY|{e!zo8o8Agz2$*qZaNv`_>Nj=0tNigS8g!U_(~C^uLUs(C29ynj64A4%aV4} zD%O&!_pO)!O_@Vofe62y(3;0#1JwC_N#?%h-NrFGkco;2^9!qH#=P3k;E+HDR%2cg zUX&Le=ix8G(*R|=nY;+l@H7D?oSR18mL;xCD868`Qb0)p%UQvr(RMydU{Vm2GuLF3 z950B))^x*RVv2G(@3Y4MgdYSJL#?I`!4|NQ^uOpwe#ubu*r)!(g%EGpdx~*hkw)Xp z#x(O8hdm>`trP)M_cLfSBpEJFW%Vp2x*U25Cd&4kpd8Ve2 z*RVa5byeA9@OO&S7vl8?7iRr4|Q(>*7M%> zi)PBuU?>elgc6konv>iLN8kh-fg5@P2zoQqs8qk}rrhEC>DbjdCx{eod5XsOAVwW86xGrF4 zuNmlRksy!=f&=;&ux?itfvLA}wm`1Z5R@OXh!3Qn5O(o0-c>vy!QB6dmdTnqtKK2! z(xvLzg`36sn5D-f++s58oD<9`&X24%-b>^*iSxti2G||(PqZBuyx>&`9q`alh!|n< zAO%SOi4GuK#&RY2g50})I5l_!bZY80d;9s>EiBVQmj$Y)YoA-!93a2|ZD@%FuuL=1 z0bd8&;%KnV;5jl(=SI6K!y4`tt>&_p;OwhqY>(XM_py(te+z_nOLo-rr*_T_Znurv z;ak}%_rq}2$nIm|g_JP%x}h|y$U~xhZN3o=<2`y zC)c;0J@CxFTH|VhI$#5y3KOtq>}I_#=a#kxuTb*D%mqel|oDt zUuB#<)u1o4q+!f#nWgJb1ud=jFV5J0ez<~7W-Kb(o=w$IE zZ3Qy`RaZbrNHO4PLfXjV0QC(iTrk280T-qkn15l*Dad_jVnDLO2gCzVRu6IxX+4(x zbGU;EzC?#cBmhEj5g1@qee5FHyT-v8(Mt5!b@~z~QT%csg{MG0RJH)nL3&@apybiQ z5Yh1T>~f|&7`jNs>pJ-pxn0*FYMC{N)UZ@+y`}hn=s(3an4x44=!3_92o)?9mYGr8 zursAlMiU~8!7TA_N(qgi!J(%BmQnLyA`QfNRtGN}&~V3#Li^YHLxhov12`(;36uuv z<-zaYpnVTHdtkC>U$(aa?3TR=E!#s|wkNbCm(E$JlP6pAHL6oR zPcE$~rXfZUd{AM3RmjL@2liwgpgg|=*Yt2FWncA#~LID~UfLFeb1O_Xa6f}GwYh6NhI zAQ3Xc1~k@eL=uLCIuC7$xB8=*c|x!wtMjypS)Bq^> z);4X7=<(>jT(KV8o}bZu@({uT9{q?b0&uAiM#&#dNg;q=cL0US#E^st)`bwVuE$eR zT3=w<=hno81f;mK zV?yd_%mwto$G-c((gRccZ<0M~yJv>g6?w4KJrdVosXS-&VWqW+qv@qLJonSPH3H0! z_A^$z7njgumJYilT3^)v*w}t)l|u{&U`dRuxmXkD(zkGh{4RtNqTwHU-~%2GRLV^3 zqwxdgr`+}wi8w(40dSBRUgDo?2#k;xnPc9AG7h%Ob|92;45!QtA0MB;@GOt5T)mi= zw;WxRPwF8X!6yega6fFmwp<79##+CtoEuAD+e)~*40Fjqo&WMwW&BQ4zY!=!9O9G zr#y1Y86HK4!8x_u;IGisso{rYLeqhD07?+}fs2WK#z!;^$N*pr)=>V!O%N~;k?%-a z3!Fr3-z8KLon%l%#9oBCLP5VVC5T958$PGFzhI!^j-4oTlTxx@(W&W`TxkO9^0pbQXy>Ry+w z8l5RRU~6U<&;7=l32$2!50>J9GT@c>kIi=;Jh%i(%wk+J*noqcTajM#cN!36EUESOFV z$ZfCG{>#fhUQX6Zz$@9-&f&YQE|*MPz)`NnnXl0&EU0`W>-6<5L%kh)3 zr*5+_;Ck4nk#N6c(V`mzJn$3cgU!V;gi;1A2Cb$5>lcWvU>Ktu$R>i#8`u%dSePL5nR746CM4l%dN?L+59& zGa2*8np2sQ7*LoT+F%Em@>^j6opjGEk@O?Nw^MX?nJZ(I^-yJ$D6@Sgmj z-5{qoP~5EG6xYT{{T<>$Lc3*5xYnHhs{nviFQ-g$YzB?{oEzU({Lp^-=feBbiaoqB z%a*lkd`ybVeyUw(`o?eG+OqnZXKpb!^;;(D#B1L>Fx6e2Z@*iztMGZV>sj-x*vN#2 z`pS}rPj-yi)LT#OtN6OsrZ&``*HD*Bx=^XbCiczey4r>7S9~l^``YXhH)(YyP{_O~ z&4^cii%8+k(a}`%6AMJo{JDR_OP}|PFR6dAb9y}R>am`S*1C8$B&jIE?+dQ#GJ^Jt zq~xDKI~^?cWr{aO=Ajl)I^hGiXG5tl>bs*_3aycPT^F2h2|zWQ(a-)`_HZKn8CpW9 zuE@NvB`|8?SguyLIj!l-nCdGsyNzUllEj>|xy_7hi6|;;+t5dn@Cw$qQ zi$gg4?zpAn)}Y|vuFR&9D)Xl`T{f9u&y8^B!M>9UuOG?2;q%;d)DJ=rYt$No=0Ryu z%m6wx2A>;ln69p_RD@5X>LOIv7=MJ@ajt+5A8<3#{0mcFX}TB|b0Gm+Gg_8*ixIB? zvmX=EoTfG=a^&t;+S~CSoC!*X32;icah@5Ol1ch3VUbu4~b8SMGxc$iXr^R<(^_5sL^(S+ecq#Lnz8%=@ zEgz}!=9S3SsqT5%3r(ypcGtyR;*}HgOX#-QEE?9(WzqURcu(>E6C1+5|Fz@lCTDiD zRECvclCEZ`g~qT;vUSK3l+V<2qt@yyu6`OOjVZh54`1+)9DKU`&)E(VH+AJ&C!_=3 zSO{s{4_VvqAbsMCjayH?H=nA=^$(?!Nr_vHJFQNOnK#ZE$#+~FP;g_^C4kI9ItGp& zZEn*s_(Y^F?<@{K_b)9#YZd4W%nn%CCh6epEQMyJ)uEFOV!#?=toH0Nq;i9$PUtjd zz09;Vqs_Qmr+yfJ?{BHZeXI)^hl1VbGMNRU2oeX$9vB$dkmsgsnKQ(UM&}j~(4hsAV=odNx$rmD(}>N#uk(s)iwGXH@DLyy1-`!bq` zLf{*joqh^fCwd3cBkC|nuLPbH7swtEbUs-P9{oGMt@U8Qhe#9yWIwL~r)V|r)zaGZ z>2~?L{n=bwrAiB`o}}7{Jg`W8q+V5}t>)<+QJz%%+NQ?l-Z@Fl`iKydtiGW`&m-l+ zp6zUfmJLUG2oE@<1KT0%gP1r;NU#NlYZu6i@h-H)4#{l<+O+aQeHWc-lR7sF@=gHeiS+aG2v^dmHg;4|jD@T!aRS0lN< zd4IEPwN<8irD@%dO^bz(!l`-(il4D}ThI%%ZJx`$Br3XkcT?bxk!{RE>CwULUwcaK zy*PjNkmq1~oi?xAxXyx&uFCuLY@`jOub_s0eJ*@?nTb}d&-B>$Q|~OadbaJ8W@dg{ z`J;}Z+!4F0|K&Qj#TV9Arsw57yZ&?O6`qFMic5F6q`Aw;2tGnFhEZrhqob-dylnTC zg z_Gx4cbmI(t3$_JGi=Lt#TqlZQHLy1c*3O)6O0PoWSI}X#XvvZtfIS=;G4|w%`^ElF z49+6r6+#&^AQOP@7W44z0d#N_{ve7afYL=QK9qZ`?JRVwkRZl@>))W0T!nOlPh+DO zA=#l-kf6X-0p(!LT@+*N&9pT4Jt1K}{G#*#ns&K9fOX}-pOpq6L&^N|wCM+s-Vr4! zgKI)h&UEzs=-4T1)gh@TD`6dT4AeuV()NUeaz*7emVxSp`Khd*8&Zu6S(e*X`pMWG z`5vuf$QXx+n?4vIheHq4)YK^IndBcRa6okVwOph%g5dQE15yV(Nd>bpLD7&lK+|De zy25@Cx)KDUT$vuVt}%nYil8_=H8WDN0}{(?Ag7{bNjkfleZ9RqAdNP`pb2Dy_5gQA_J78&q}&#BzdDf(E%fudKL7M69kh6hOo?ClJ-T=_ ze6r^K2T6s0C_aJ&pqoh{|Mn4(;puMiAz61)DP z$L&WEr=s?Q^TL5$ySm&ABL#Q+@vRmRxP8}$KKz3svZCBv{Q=vLhB7Ef zFiBYIK;lH((D}0C{a-zd8pW9kXNq)wMUdcq1PR_jkYHBBwJat90kaC5@W&C4Lhi1K z@UW}VlyWn!e{sGxb0oAW!QxKe=eCHn)$Cq7l2q&3AvO7D zSN9l|RaHT9B>1@{``OEy#srP!2;lCQvu~y0#)_U|u>T*dWW{>$0y(fAB(v)+bA-WyC!=X3Dfokz~qIwYe8D&`=6|)fy z;;Zv0v26olBIp zS!10(2?LqfOJx_CFS%bxh4QrNTqi&fBUEL^b1n>AE)Z$Z$~GbtvcI>&V6fEwBHxM? z)sO=!4!NSRtLU$l^Lc;Qimb(Yew}(r);rhW$*7S`IwN{7_@0^hM~_-D>0(e3Avs(}_E- zV6z6|>MFsKvgZR*tU>q362Eo96^!Y)H;5p^HXcf)!;JN-fz8WvJHV~AfBDime?Ox| zl*HI|EigE;^0VcKu{EnhnD`a!^|)2z61vlxSoy?!p2@6BJ*OiXYGSSZZGUXW^98vE z((k_PGwpp>$hS4(N{gvg(~~=n4=!})Ded0l&TUc?UeVCjq`B8^)W0$$Z$(Gxq)_;q zCtOUWQheu~N9?yc=SWPoN+tV;COX z;geZde+-rK)6z#-7`bu49CAB} zZy+QF9zC>6g_zkHnPy;)1DC zFriaSgvf|~7+Nw8#MGgiK!VL0zy?rC1wo9Kc2!OP2NLwkaLDJ)pa0^K2tYY_vr9{N z`?svOz@L*3-!ncLZ77viy8h^Jk=UJ#sFH&2^BLl7fM^m5iyn)JD1>{Cuf9nAP)d%l zGh8Z;CnIsH#AY@fw9k|5ubv=E-m_#U~b<`N^Ckv3{?#M3{LsqPb%GXK`$F9)ZO<|0$1Gjy^Tr`x*pg3$z@ zZ6M=_%FwGdY)a!3E8|MypGf4qOk)1FVRk_1mk;H4oO_+M9TQr0Lce(TDyG};r;qDZ zUXy``kF}ATF5siN3=be|wbrZtquVBFh z(g8@g9si--ITP@!Q8+v%-RAUK0EiA43uur4GDmznvLk)HRytipy&1%4N6I}eF+HL{ z#sIzj0pbiH9n*3Q=k5@lkFZfd7)gs33EG6w;X_}abE^$m*LXZ%pom|tHtocMQX{zi z@!+m$OWS0p9B}&x9d{WXFjT`>e@k)@07kSf$Slf4*9GZ*P^9OrzjiF1wQvPqW{<}1Wy)VfDD`w9?4Vllrdo72{nd41N7v>yFNnGS~%nkn@g4E z9|}C+ok;L7c_wb{pCSLY-q$2ad&pO(F#XTB3& z0zc_E2nT(~59vsAvmRv1N`2UO_9^q#29Q6%OnaK_0NTUWkK->g*vUi|Qe0Zgu&Eu7 zHlk&#qGSfHWq-epNKZ-0*L5}RQ>{o6R5`jqF6jN;L19z9WoW}Uo+D|KjxXAmwXtO# z)#H80?fOe^?}?u+dY2rdmV9h^%y(j7Yt7B$8y0hQnm>9d(Z<|Oz3v09VN*Ab+U&M;E2z4PiRK^iX~&5Q~h@fcMB*fCPng_$o%!UyNY->$H-kge+7l z2~u>3ic(+!#U{b$GW8wRW_STwfDApcB}|VX_ynV%e<*5l)d#{}8M@EK*8LXcnEix~($++nrTXhKYZk~QzNQejS zJp+Tp?4Zes38d)FT&19((8ihWIQx7ByKnS8ot{Kus03*t_y&VZ=y^aoEE>}`tol%HR;q}?N>zBk)L;POS;$WWGy zM7gcD`?_-K6XwKODp!Ol_;s9>xx0Af)jE@yQNM>*19K0Fbx&R5=k3ut{WNGSYT<;j zz^H@g70K;miJA8=mRxOX<2abayRz)3e);2j-77nyf+N)mhAiaOr{AW>LGE;5XRXPS zfcvGm{Q%Q_(^H>CGH?KY`I8T-q!J~noXOc@p{G{c)5#M~ia zCG46q+yaqwq;KC+jQR>-8)+v1JxYMKAo@s>n3CnCetdE^2gbcQxe_-(cP>2biX2khY58w9(o>^$#3=hy0{)m8S`vIPXUL6;o=b01=rp` zvC6d3&`a>BGe7AR5pGwhWFjai7%4Ck=a5id^xLkJ$!eiVcAe@*^w0 zO&OUm0LjV8#ZIQDr_UJe)D$bVt_cNths&Rkbb@vu-lZTDyh$jH0@cPog~i0h#W9?F zfX=>Sa9RN4q#foVs!#$nM!`GdM=Vli8%%?B*yKJI;Q zXU@01zGSgtYP*(nzmn$tV|R$5i?P|RK5)V5^ozgDGk?VXbY+HrH5Y*|#0S{fJxwg! zEFlp9Qeswrwv0cY+=oW;LKD*oXlf95Un=%(YuU&QI7V_8KGJM^Z2YWgWZfn0g9jB; zAP}j9*nH%o9q7+yyQZ^Dlx~btRd+{?o0MgEGDt{Ah#QPIv9*Aq9COWFt5+j8RopLhkPrRm723WE8`Q7u zt4;XW={f@P+ktzp+V*C?fvtNc64)Zbp7mhMnZLh3 zWxvMumckOMlmZVTh>UK}<>vz@*g;>G~{=8E2D4s{`ZM(b#WSP4+>3fW=^yQj@oT*!3l@2H{*HEq37?5$w^7 z`A5uR$7^SqyZoqIT|2Mw+AY`5TZ3x=%*LC_>E?8QZFw(qNVFWZS#Gld4)gNlp9Q&n4df-@@2D*H&rmV(^#KYuIrLkI2rM0XCtd_j?cj8o#>yw*LMop zLeZ#S^NDM*usB%RRl>sM>Cv**mq9C7S8u7OxL+jYA*1(KXfOr?CNh5@Q!p?(THVP> z9>C@Y91h@mfV_7jqyc0DE#d*+HR4yHG}4?m`F%ObUxAR$XGKE)WXy-I0d}_KS77w6D&Iiy>q-sT6&N$cAe45!XOcpwXZP z59p}VRjKC%Pr>^)C{8K!Sx9Tq&Vvp_>`e4c=6%|GNh>C2S11JnqYp$ zn=2-|+)v{6SX*_KEyh4yf_Wb66EhiY=kNy6i@N-lC{YB6LidfzxDL&^L==D(z^kB69c^Gu=8WyhBIv)&dX9!?avkyW z2w^)C>K);Dx?EgBO76+gV>P;(Z#7DK<2WY2fDSPR zvDge#r~7#16h^>RC=J2EoQPiSYqab6Hh71R;R0G4;ZVcPUxAyepoc&lXyOhJmfGYa ze1H2I(Y{`V@C8~iN@qTPF7wV^AHl?5C)@Lb=Z6w=&q|YH(orQY-g{mg%Q`Q-<;=UP zXUWj;PRl+w4lRtIT%H%5o_o%5^zPgEH+rTPSB-a{Tn0YEu0EvWQqR)CN4&CnTq834 z?Wm3TC($z=lJOUoV&D#Y_t^};6{AP;{r#68TL?X=obwv?9Qo>r>)ZQChWtl(2&8=r$#fR-j$|V4_I4xzHsMS8lJn=Z|Tlu&^HjKWqOfA%f*Ik@w$tc z(w8nfvf%{~t)xYt)bdQ1rn-0V_Y2;R-();+?-#8xrlFWtia{{#)h#30=TV`v+OsbT zg+G>#Tz=nEyd(Bg*c(9sfiP`TQ<|d{%`QTYNIO9hRYF#M+WG)d$}9MNRyuYqhR?dJ zy4nPNlFLvrH>|5*)CwVbwvy$e4FxbdUBMK+Oxl7DD9UR$9Jshir76SYP>R81w5=Di z>akLkW@ZfIpdTz(dsu~NijThIqj}yTKvKXgCR#Ik8X`_ z8+d?oxnn(fF-xA~U&(<$%i}=K$!?5Sr+)J!`?07Ck6VIyh|jP68uaGCGK2mjZxB8u zt_x{hOF(hg>$8l1$oHtU>L@C@a5{~ZMPzh#M5N%zgm2DXNw!&VM;I6Sv2C(w5LIYx z7|UsMo@18_joSzsVWj6s)@DKVfzPT4t?5Sp(iI>d!U+9U*$ZLPo6GPJ)|H_Rjd&@{3o3uKbtQ|8XrWeYB6tbANt5aOB*zzV$ z4fF@w+ZSJHN$s4f^?YW;7@5n>WiG6;Yr!>KI0u_EmUXSOnT}bw3=G!qzXEprRdW*g zPe-ru3MBUO{#(3O@qZVu^?$;z=p5kQwrfh?72wfrvH#-iNjnU^IXQcTX*u`CPXhnq z0_g9Y$6@MqPxyCdkKKRg<1zjJ?pJbtf$x9QCN{IjlK&$Y;DYn8|0|wW=FqS?t%eS~ zM?L3Ai!uDYB;2!+iUEW#Fs=ph@e&I^}k+{~WwoRq9 zv57a`MLDg!kqIKV{^%($v&=I;`M*X!^O4EZDC)?1{PFzsW&H{3Rg-R0%`exD%bu5H z1lHh>T6Yz7#jG|K9MvL-qrpDcHfP!@H6a)t=stCLz(axGlP8&pHsAgH>BNQ2BS_0A zAoWId_49ZeG`-issD5{w(gQY)dp|ec_QV4W*huK;+94OV$C~!qkVp(#W6V0I&=$m= zg>AhVX~n^!$KQ>luM%RkIVxSaEJjRDCFb=njiJH8yP!%YSO5-X zB=<;U=p_q?Jk%3G`)d=)#ZKf#z?q1ya5276P#r3M-K*+tGP$)YZeQx7lWDr0*Jw5hb#gxSMr{7j^3`{HM<&|~ck=~$pjrZE}uufT> z)6Flh;*qqA9T)K|d+n0k_#rGj&yWJK=j8OoLlXHR=Qa;>+{ZMBU~(oeaiKHrjS zI=Co)(+7uid(m16!Peu~EAF^Ri0)A@HyYL%Iy7(S&dw{TJEpfL>DGY!xVq;6<%iJ? zO6>}Qqlr(#0-<0irYQP}R0VMVlOUV+sNuble-?;Kv8^vH34ma`9m1_^*FIdDyYeQk zKh~9s887p+uqyjjoOV|mbwC(z)1lEKjVc7n^yh(<7lKxvCMA4V2r4O{5E2klv~(ND ztKI?b=ldtMAApTSPZ2}VibWC0;Ss(;x9Aqv4~swt4fKdMi_=O(+&#RZ;W-VIkAWx zL$6nvVcog@M6OHjSxB2pAuoW2=kFvc^u-A<{pLXRO;Omm4aqNn@6Qp_H#x`T80+tL z(xruuXinYcUql<0eK3){4A=9VgZv&Zde39rUujtp$Sh-RYx=jRC*C~e@qZpB` z1=WFk)F4bhCPhi>VJU;DRpmp-30KWH|6iur3xvj5(_=* zj6PUJ$nuHVtPrz_5zJ&4V)ugMDTvQ2$ii!9rLIr5L3HA$YR7x6diYIzVjPKuMZN38 z7k83v2;pd^@M4XX6yhUgU-9tp})44&WBO&0{Xu@U+&;a|GLTNwvl zmR{N@^9#QzzV@u;=7lf&4JHNrLh|_6K!Bex$G)EBFXNr|yK}>;S3%|KsU-CA=R3Q1 z{d*GJ(rw24Mv8Wie-!W2dS5#cG`F=yn~p(FZNb=>Bs8tMduCLZsgxMzVXqi{rZtdM9cp?{7P2C2lCk^ zAB`Df&i7tFa4Nrie?5r*lG0Kyas^`l4r=`iTrQB`orL5Gn`JneKmx%FdKB)_j*gD1 zIORAXC_M^|SX^+ph!#ZGgSlf}{e6{{!R)`S)>W(RDTEhY-fSE1f?}z#p|2LUo*+ zL6_2zkithIv^5vQ5XMO}(9`%aj|^X2{>{-PpZ7G{pLO0GffDmrSbz_$RGgj|f%<_SI2sah$7j=Tdo%E&grS3EQk!-v&Eco+mQZj_bnp$iFr zl+V2kn}FX&k;J;9fX@D-c3oLkUGmml8D-TMVx0yG0{-fb?mpDg6?<>fZi9<*da^dr zkX+B%GBOG#nKFcvFbCy+v$nRTWkC1e^)|l5YJzIu5ljsbq2<<+PX)mh$m|fSv}U4l z8wNx2;)3jlG$Aa^z{wlq)AuE(Zw`KG$qoaoZp7nh))H|+Kt=29p&y)5oH7f3%35`o zuxUIL5h$*?`!r;1>!-DCyYCE^Z`xgElkvxaHGj^iu2}3+Qr2{2%g&4^N7lzpS^Ey% zza7!`OuE&ee_T5@D}4csB1Vy1-okQKEiN}ylvgGUXe{Pd9DU|?=VgN3`U@4Ai7LI5 zaq~bfJ3mcnjIHt?*LrmG#Mymwon{TbP3CAmlE?|tZN|$qBc6?J$`$iO%vZ+T&Ux#WF2wefF_&xCYD2)S!xk&IJa+Iub#ic@3S{ri%4|D>^ z&Df-n-Pe~DV-%Y9uS-IhV=2`cN^to_@<2ZA@U%8YAq+rBFl@Qn$28NprK4tF~$MJr8MNry0s4AqRp1O z0HG;a2HH?!?jXg!f*IBM>G9BFY&|5|>l#(x^#y@k58a@HQ$&v%4teV*LY7O zIc|`Zf5c@wfBt+(P}l&T!r7bI;kkV3*}nCs5KX{PukAxfxv8n?|FsDus8ihBRby}D ziO!WTBEtgFS}d^J(rWcQ$_vG$y8+njoQ?`m#x1p{+mXRemyafecKgGW}-%)|S} z10uhi>QGs0<8LN=S4X03q_wt5wR(`LcKq%d-C*I4Li+$xjpE!ojZvLc%;3nS!2B`+u4=%pPN@*Z^U+XZ0nMJLu?9{H7MW-bUT1yZ|cLn_(`mxWB-@oU*r7N-)zd%0(Zemji8r{bo zKLPv^0JerE6Yjur_~nRcVvb-3rf*fNr^9L*$Rq^6yGklOwyUkVQs#+ zN?HWKO$2jjWq!|W_2r&*IS&{M=grISuxdXquT($cs6E9vi+rfz&v9Yx6-@>&OXZr- zTvEMs!CPL19>S`RQ+O&?Cna z3+nQ==FeUDwz#RET^b7C-uD@-wGaJfw72lHcUbANT6E+#l|IhzkevTDEp9X;w7Yd- zzNwT|^pN=I8;}(I9wCDr4ZQOks`P?J-Q+(yBB^Ke}Ax zT`6nq#3%~ptG%kPS(^oMG$saycOR%&|q=%*K6~YUau1p zVqDIfh)nP{&vMH|9#B_TwClMsymQ5rjfd`T!`dIlMT7-FSbN#*QxEcXjS=pMjvI@L z7NTLISW<+sL{CqR=Ec^ksHiyh3o6;Ze;{=yKK@2fP~mWQCHQ4>(?dKJgHsXrj1Qg0 zDqM#IbPlg|g4KD-)^Ej%6&Jx?>GhJrKoGWdh|oKA!ys10e8hyC>kbMFt->USc=q_u zwyrojAtZyxgvJAX7DV4)A1I^NfOf+BbCCCeuzvc9n_0?c5|P*(4HrY~T$V!5^|}}= z)(Y*;?a_WVkvW{akFC(t`w|<@hID0q<(vfbcF9ea)rXS~&Oshth&(&5x!Y;`3Wf!RV$w_jix8G1c&T1r1}{9B1z3`ohy~$N8iG*R zNH>&We*?B>?DxwP5R2$KuzcxOXri#4_Chh6(i-&ZA%>ViX{W8zfl7v81l6MV< zdfeQ~q~1SuEnbz|5K(nvZ$y{8XzSS2s&y`3^mIDDaxMV9@weFmDh`p!?)5TpPO4{% z7oCvaeaA|qWjnkbaRYn4yCfg3UXb3d{^g)c?wSZCFR3(l(N{@|!&3%gK3>L*B&B`A+>NI`gPYTDj3UXSP*^6EjE|rTz7eH7v zzBu-6h(QJ=fl>R@wTv-vKj#Eit#U`e#f_+_O0i}>YtJPjZ8ybb_K$C6 z7@Y+V8J`1@LU}1T| zB*WQHmjRHNKu}lYgrqQ8=vj(T3iSGj<_6pG>N+4AmM=oW#8e6Z1%tIo0;F{PR}=+F ztY9Qi2U(1FG2_peF$4K!ot~}LTPkhe{VkPAAY1&u!pv2_F&3_`nU|0%yJn%#p~{yLZ=vM0EtRV8GXz6(AQw8Dcv*soyLJA3Z0lJRB96-Evo@u#7MyW5((o8(Vz2P4g?eJVunIS_Ljw>qZ}=By^7lJ&;92@_W&=gv1^-5QtW@>* zpLsb)n^$1Z$x#Hl_x3OP=POk_0&&w#ydlmsFhSW0|6Fsj*dK|oGyeF)!ye7D|3z+UAvkQVpHzx`q!|M*ykQyxX|g{!+a&NeU% z=L^rtH!6`^A)Y*}8^NVRvdFH~J8X5KI#x2OR%UP~uHlbZ1 zSI_;AFTCr7iPUEvS!ZS=p5y_x!Wq%Ll0ki=w{!Y#XLX2d4`-DU%s<8C+3P=`va8Fp zmUlYXrskUp4#SY_!?woDom$>Sl@_l7|^eP3SUMEiDIZ-&3|ODHyR494Hn! zsq?`jlUxz4#u)dg>Z{C`V|x^@_#Jqa^~$zHx!W(AyayDaf=cxM`vF5iD!^CVK6s)PU6cRSxEX-mv=ce#Is|9i~If=@dS zEo8Z=>9I{_w-6GJ$wmh}VtU3ZW#u^B3d8W49mMttolB~_NM%W7Ix*Ji8oYh!xG*Th z9v24isx|WRk@yfLa4AqS1o(U`HjxIJ>6^WC(?Lw`omk?0{&_3Rx4_%U`G3y$Q;97^ zBSf@QauuweW5{pEMwUgun6ND~6743*FKJ}0m&g5b&iAO83ZwwIyuh^^tOo0V28raL z+9>d(6TsC0;fu)er@m++C9%RAA#D8{+AF)$-+z&{VD>D}H3MY)Vp;xRITEIlfRSqo=WBc`cL9hD2WD_Mn*;RuQsMz@r05`LNhVVsc8B6*;TIkT?FG32A7srA^Z`S< zjb>4vVW#)cF8OWrZPZwYwfsI2CtGW>FJE(v>fN8hIwG^e%q z-&@!(p7DI&{MfU+HQ(1nR2jGh%W26wAFMv+7Mx-q+m$yNQ+#DCO4DkssK3z{OPi*r z-dFW%q3=SQHJ45rHj3VW`u+w0t3|xLdkqX$g3~ibe~LU9YIA6#GG@IhXcEYP&4klz zgsb;~UgnZtyC87@NB~}6E6I9=;4LgbS>}%RoRaNmImUto3<1K?DE>{+9sTp~e}4 zJptZog2o^q>g?`TfslueS#4ven!e^PF^Uj`Q=tR&!rXSuD}o69!^ryzrf^~+pfsl+ zfv$;3f%MQ z($kCMb`Oqh1grs3R6i!+9ZIu{q%f4%A!8W3)>yVd)AY+B3P~XKMCcX_@v&g<%M$of z0hhrb<-H(t+P(U8snJZY#(bj-?c%Jgo2I>9Z#mOB&{>>)pMALG(JbFA<;yamD^>~r<^HJn z;*WK!uY8`le7?EkpJBtswl4Au-z~HqVD`S+yb7B<(2)_|zJ=NK=D%_QpuCw`lg$rG zu9EBkU_Hpu58Toa(n&%nG0{P~)_{GT*o?sk=MS1>w2Pj$o@T9Z6w_K;K;*Q$4?4@? z-?r&$WGvGZOoAm;8&+P)Kha^oLf#~04p3?uDnJFM-j7J}Lq)SKo(JQ))!s_)vXJuH zS|Rb&!Sx%td^}(wz`+8_k#i=?h=*d;@%%o!TljT7?L4Zj)!O+Z^HTXK;$cA;zCz6g zy#`8LaDAEFWo%K*z6*mFr0%rc5W-X3gJe|t*i+WM1HwWT?5fa9k&=`IN;enwvs*WB zGUZ>eT{dCu7r4>v*_D=_e7_Z5M(tKSaCEXqDDQiZ^P6 ze-W&j!=7Vd;LzBj@c0ovK!E$*v~mV`FqBvn>jG1E{yGMOa9lVMTh4Hy@=n>?U0$yD(PeGyc2v5Sk8+)H8!DnlK$KHW zhWP2kA3vec7GA%e7L6vG%e0&i>SMl%x^4|-aA{;1z-5H^+X67_T1s8rfyf4GL5@9Kpz@dsxT2Ebz2n*W{4KH^6e1O)nSe%s{ zM93!5ijXv6q9wPqsSN5Ex*Lt~v|zo9HF9*oB9#Cpqm{Z8=?Glh!}SP4uHlYx`Sw<4 z^V#>oNZYJ3W$+7-=p1@ud$v9)#7m-#;G!bg8AmHBE_;OOWGQqaF1H>p2Ee{TR#_fE=2 zL4~9%3^Hhvs6oV;dQ>F5+7y!_C1DZ{cDU@hC!&|gdF(Ue`ge?t?f8O(mcX#EQoJ}GY`nn33T@s3 zpW+=D=W5OGaJT=3p_cAivR#Vrt5q!C0q$YWkG4u1=({qzse&3bxtIt|~2(`Ji*0f8? z=GvAPd;DW*e5XNL~0k*8wqa zOdvj7z@W!J-oC=AgcUgX5pHnFRuJxhL!rM@ByK4v_I>-VDeMT| zXocJabbfp2fAQZh9#u_^G-bHp(vL)?~* zn9F|7ceSMcPW_XHCBBVD1*uWuO-x?ALDe1ae!xgLmn7^E5i|V(W)mOaDRd6WJNoZ* zp%B@EQbX7tHYA4MPaMFRAmFm*c-dq+csNAy=+)M(0Jwn~jHeDszg9~}|8$NXKpj#+ z9a@i$tG`ALxG(w?3+dyxmQU4ARWqQxc6H4?GnSL1cysXTdY$x`ysS{0Ynb0Y1nisR zz;UF&D|N|M--{W{7a9EpwWU>^*DE^DNN6|L?)cE_vSGG@PiCTxMVIc1VAjDNzkBP# z^Ll01&C2n9z**#0a0M&j*5|gG_Ge|rM?F*D)%EiTC|+pmwV-T9ZCQY_dk5VfEz=^; z6GjoreTFNUX^)!PnoXPh@x8x76TJ23$2I%H#5B+)o4>p{1C%crFwts`N5@@2mL@O& zlJHQVU~7&eA})d0EQ1zhr;W{{GmU@ahzQFc16hOt!S8;C^LM%3ZipFvq;^bmS2FE) z0J`%)KRXm`9Gq0!LEdKI$1}-Z_{0eUfVu#D&j~ME>SbG0Kfjgjk zzeL*EISfKBN;q@0ob9~%)~ybaH*U0V(7bBaghw7dFr}R`SdSY(mqB0)(&r$Dsfv~h zLf@Dn{JQT-%jx#i<+4@m3=hXTe(r8IR@Oqt?PE^QMclr#o|BBQHY{KbUm;lKIp1dm zA3KMgm-Dl&5)wnvu?)BVITwDIJHc|_O>LcIj_gvkpGHHHBQ3F;`yX(9?sC=m=l2(4x}obkwkUE)>v0@mRSpwS=e5*I?51Mf>fXlPG=YKtjuACK5d zL6!RuFhjh>v3$ADZ?7D=qtS$uk$_Ch*hX1aTWf{}9n?0QvROV1I0breS0>sIw4(g~ z`8&6XTx#vO{%Fzq>^XDNK&`WdU0f*kjvs9_u_lz=w zvRr^jU?x$;87I#H{1nZoU+yDW)mY#}D_c;0aq{c7)S9LbVRPJ#*w`um_hSn6$22q3 zEYENSoKZdW@byIM0H^VhBB}J!hhNVf3~220e?1-4R+cW03C`MJ;C%X-F@<}fhTr-R z-TPD4>*!TfovALq3a-(yY<{F*Okc*QpN9ScY(_EM9~E$mRN@mS7zAwtadc?JvO_H? z2{&w0BIqwBN@gOZesV`REqtK$B!luh zh_;9#&(Pg~DCq|FQ;cyDA{4S*!;QpJ+yV#wuX>}7ZVPiD1O0Xnjv%QJaCQ;jQwFJA z1BwNiGGW!Xd#H`fW2jFUC3joa0F;1Jqu%krH3R6%h%KQu2%OC=`1fctNV2gc_V-~e zrW!0ih z*B74{SC{7SWMd7ol~tcBJAQbwSI(<`L_C7o^Gvu};cWK8vG?aq{fs6gSQD}-&kY|W z?PDS=gSK6$`mVx7etT>TmnfklI8i53SGc32r(J+6FMd6Mj|tXm*@1MsX$#P9B=oU9-<#YBns5;1& zos<|W0J#kgHhmZQa=vbWMF_yh%emI{>wrl#eqk@PcsHmnU~@rKFXIL{6z)haUS4mb z^)LS3&fNS*f5hbMBZYn|<^on@8;@Wav16Tb**@j;{G6fuNXp3TOx{gZ?qTID3^&sp z4Fgucv8iirJMLeZo1%GFK;T?L^pKW53^_QB&s2rELjwXX0RjhRJe{~-hp!DGh1L*QXybN{!3z{i(uXfl(9`VPm9e=#2hh$EwSXfJoE0s#%tY_lm~eZxa*g< zRapLBf!(+9bY&gz2p{~c%5QIx>H~6n-@${gRqB->W@$9vY_!0QZ0vb#mQp+_Zg;N8P&D#*G|`kp_QMWZlQ+AiKVhCy;n?>e&g-zKAhw(?FNX+xM#j)1Yv*J?iQ?x9wakzql6GZz_EAWVdeG#%~Lb zf3xs!TU^p?@FcwMK9}*z2dbvAF%f^O231Fi(I}4oetcklF#?0PuW|9u3gO7*5g=oV|5oa3_r#5Z8W6 z*wfOtV|?#5Rrv3rA4=;x1BL>K-Mv)T>*^E7TfzTsQ_DOv7 zxwzJZQ`}N}N#dik9`Bo6)?cyS$id^-rqO(CtNRvb&9py`FWAdF$00)F@!iGhu<|ke zFr1vz)L6}-+~&}#UOk1wm7IT_NFP4LCR|dVeQ-=- zYV(Y(21RjI#ilYJ8#5H`+w^A63Ll%4$Zd*Z-?r-yiP--}-Ft^)-S_|FqMfJ|Dx)+N zvNAFnS_*}-BZOp=z0#mkNh)N8tgIv}J9}3Wvgb*%$>#TXbzS#;U!VK0&+$Ee$or z>ufI*uiV>LA17X68@b7q-;0X|E5{pq)I&39{)IYq0f&y=CD zOI-aG0TuK~oih#;mIB#T0{C$)6q4gl)^R;3nAUA|d#(|gC{~>BWB+{3Sfp*&m|*m# z<;Ut(bHBtn@Ww9LK6EtWmQ5_~;Tdn)#WD08kOZz9D6C{&36dxuDon-Lx&CN?fzbeF z9S7J?B%YsP)q~4CA6g=5S#GC-N*Luzv2xpEFpoj|BMGvFEgV!8HTG&a9J_w{W8l69 z=twC#N*`C<{{8!}0`5pK1Yu32%GhPG&HY_sLtrE)%-%#30T>R>TWo;gNCH3-3MuDC zx;zyMB}jTBY%(v&7LFP23*`A!X!azd<_usr4HGpkMhswl{xXECKnfxPQqNz!xB=iT z34Oc;w8EW0`T*|~AU-2cHLMsYe$*!+lAs^(;P3{(wEh; z8)|fdjG8hbg~g0}27eUePdlV$;uL06I4~gzBCS7|>DEr(ZARV^pdys8(U6ynWUb=5 z0_x6Y*S?JCWa=GI2SkMMq*a9oE?lOyiJ#(+nG-*|=*_+jh`;_rl0cmb z7xgG`|Myr@ps*qEJCY~A0n@#h#pGc2fbt6B2@HkdSSl!+4H+=HUyfx2pq||L2!MKh zeZ8#r$NGDC@~($e5&$k0?6(BP z^HEHEV_JG(O%cXoDNJZ2ri*G}ZYl~CBa4{Lqvy|`pEoTjDOr`-BIn{FiA3+|X_3V9 zB>xC+RI~MBc5uH__sq>lC#x^mhhHaZ0lPPW)MAOw)=wu`4ldOvy^X!U?%fUB14r3J zs?t5o^uGvL_kG>*+^~%mjE>)D(c8EgX^pnad5INh3>Y7MRwNedHKOKmfIJd(`)6b{ ztnO0Ol)&$p>fr&dGOB*Sag!C$nbbYkJ`08widv*(L~O1RvSqZ*Z3?Ik4B2EKW1`16 z5npws?i??NP;jL!DwQ`+Xi73&%`PeL^KlE!)|812UWaMBx`I=p*?MbSU2Rg6$5j}I<3&HgldSqV zP3W=df&AF>ok>A4Z}u86ZopO@b0tcMS_#4uuH-uguWRfc)*m9$~S?Vi|xW6|_PTP6RwNQFDW&A}I~vy-2Me zK;kgK7zx%Sjss)1G)i;%pKDNIhIKC2$m^f+_fl-LcrJ0z;y7!c_7GE zi+IL?%e&TH>P;F?u_%(X2Y&uQsa!MIjInTq%#2fN!DHhXCrP-F{}8lr#X3eHex3mA zL!8FS_6+*MtTcwEYZ78%-ei(C`t|&wFUbZMKun$_t=8}JTia;*KA>th(gpBFA_w-s*^ZbPUQp^ ztQ$1YYTtnt=qpAf63UEQniLm80?7(*X<;| zw^#Kz@zAkLYV0T7P>+yXkJx5A)j^`DCgMB@Lz9V)5H>mdVUpDbGAHNb_P)@Yl4DLs zPIFMAvg8H-kn4h`bsMv?Wmu#uC{u5Kofsd=5%uGvR|>y$OW3ecX+A5vH04NXa0N(HUz=I?THo)e>WqzLybZN8cVKyvx4%J&_OY^C>*%|( zY`l`wu0Q5=<!e{%hG1*vr2q9QGSiSzj!v^7wNnN?ce!zDcj9l+Z-Qu>GtF zy6maT7hsp}vOl8cS8Yh=1qO{u&$(R{?RrA;i8?!MFLUR=Z81!k48QS$L$I_Y<;i`G z?Weui9{&Zxk&v%@y(y>^!bgs0@(IxUuvU<99q9n$sgt=&3tr@o0*(c)dCf5Z--P!D z0J&E{0MHD}?c2A{%F3c5n!%&sI6wg&W@3CIB76KE@TA0^sQ_42$h3L2$dZbP`fd{d z%fMd#>Qgv|_F>2k2(bW4TG?V+hlFu`9Qg2;{IjYq_5<#P1j;8ODgbK5uyt$y!>SJR z%??79H1y)f0AnmJE>e+b%M2?DdNEm$uyg{il3@3TI1|QkGO{B3;SJCY6)r~br>D=} zACG*6unklV$$&^rQsE*z51s&ERw-(^kWQodG~>JIK?wYDc_)FaCSF={tl;|+^9QIS zvipN3A)-J~TOE+Z6^E)a#Otcu>=Ud`W> zSjZaw_Ei1JdcF`WgFtVhW+4p#m_%?C2idj^xK*DCzEEqAaSpQZ z$C+9Vx#hUts9j#zs?z-A6PxWlP8eDqOE=J49k8zuc3H? zm*&%9F1}m7zP<;GiCpJq-$k4hh1kRh-3x%e0|;ntp9`mER{DTra^tT9pMss8e`ojt zSo(#$NE~b#z3TBwP*DRbE4aoVq_~3{{UlhARtBExgTNGPXeHKJo^ACrNXLJFv7_`F ze}=m$xFfPG5iIU-n||^aB-z#vPe#(cSaICdc#6%*+hdypNn58>cb@Ywm%R!&fL=#5-hT^LvYH4%D#SMxE}E?njdXumUA!ud{g z`pp#gw0(f6!+{Xw1}&MW2XR!49J{u4wz{ccYVNDywvCm-OXqc4T5q-vI3*8cyt`r` z>iEQ8c7rPCQ)|asLzM`-&{Utgro>lew zkH$_r(2yBDFV*2GRS+E#F3%2RH<3VgVq8Wrz|&_6(;I29 zIPno06X~zbu1gCtK}WXWok3|Rs!c~l2epPA&x)Ja-I{7+ZLJ~VQ zp~wkGV&|;HfG zd!5Jx;?i<0i_@5o)ojSR*RH*TdK}=thcnAZ7eDG=>uxNqPV2(gN^5Qx+&Y!}#{59Z zPX~8nY|#zxmIMXr*(s$lP73_%RFV$9pcf8&eI}e2^s~vn>KKb(j;BRwPR26EvXp$%IQUBkZhYYe|Yg9Z?#jG z;cBvYrV)GRP+G)Odv6XapMdilT={*6T`rUi@!nsUd}Gu{Ieq!Oxd^w{<;+@h=4wg@ zoo0H-<5fHw8XLTgJc@KkZ_TgP>;P=a8*V|g0d*mXe1t9piGl0#>cW2~632?_``y_4 zDDZ;zKv9J!F)7;w|AQ1^a&3|dy&rw`iR}1%b-^)_xJXc3L`#pyV;{y>5ZHvaK~ad! zaO-2RW&jZIr!+r!)L{gHRHzs0FAxD_xc2=t+z!+UiV7011;r5bTN+UB14 zIdF-rCIAnK^9UJ9(Br(wl=9i$3?=H7g3gMRL}*p;lp*O#;8jR=2Z>e$AYfyAox{{B zsiuh7p0EimgBV5U?``V#ZEBL6P09-~btX=a18vbFl&WHmlRc3QLUX4S`+__wQ0eOG zJlArHva2RacekZW`Sw#%X@Z{jOAac3@};O<>KuIGkrAOmb)i}HiLLU!&pmhAR*#Hw zyw#V6h`{IR%QMnTx$LH)gYgl-u{R6k9+g|_-O}hH=*h-DSd5KXy?lkaW6zFFUjwB? z2m(hI{MHK(veM0u&Iql7rI|n7dQbgGrL7XToSKbl>*6f0p{USX_sw08A9)($nnycrYS@n&Vv7NZ%bRDbv_g@+Fqw@pGbYV3u zJ@&TN)SRkHAPlvqB`*Jkzfsr^*tbTW>@~5}j6T$B5@}j=>~xDQL!>}rh@p~OP1@e^ z4tItRzzV!OnAp}H+5I`doAvF;;>WJ6wBFV0W0z+HQ`En$Yv>0E$UTrf%`0j)cr1%Pl-4%?42qMEVNB&!@kHwx8<`&L)zT0?>XRSULc9BpVIi z4xz{u_-^O%6ag4T>IG#2MZDKA>?MI3;4@F**xia+VeHyHB;^ix9UdyA`!`6S`|FFG zv_b(@F2+`S3MF5IYW?_c*qm)(iu*(&Fg(yhIp$bGv!Zq-yq^AOt*&wxrrvqV&w0ljPBVf z=f~TI=sG^uPfWG7u-?93dO0*uUhuiFxH{CxbRMB>|1q_U4Sy{n0biHOw{p0C{_h5o z|J(d8|7Vop|D-}sp8Q|E99=>eSn48?B`Q;D!#}Juqbm51TcC3k3Lz&O{%;Zz>4&dz z+W(WmJ1PF;A4V128yp<}hyI`c&CkzhyK2Xuzk~cweVUo+$Fi248dGYl;(g?`Campw z6c(CgVRF&yf9XIHe^qd+yOP@015yXyvJP|9s`d@>{_6)!P$m8uLOtH!mV0lZOfjvBD}jjb~)EE1rU=qhin08ZGd##U2M>k_))sNOF| z5mp+O%MMEwZ+X4K5#Oq4vd;9A(risrmDgNt3+uMwIPp>_o~cxFcHKL^?svK<$41c* zKoh393pOt30H_y?tt%%H5!m)|^k!V&4!A7>k0r<@_?)KAA@{KW;VEi)m&A2q?f8tc zPqMM&kW7r`zJPa1i+r#8dLWe2SZe@J24za^!Xpzm2MzE*B5!YD`En1IZla5^^{|lo zfd~NPL7WXW#IcXOCO@3ZRv;3f!x?x5t9B6{T822=Axog(&;x*y99_V}4-*bCXAs#9 z(z!;!J$@8>v;8ahO1sVn`>sPK5`<`Wocjs(m4*%*BOu9`~CpMs*5PqOI=mGKydE zT5+Mit3cEG`X>2jU$h1(b`Ck}UFD_889BM}mTMe@655NepX^rXp-=>{F#}&AZ4XG* z$mlzQ=W=HTC79@Y{A84ERw6W)0uMqI0_7pDB!yE*na{`C0CI73OG7 zC@-*m^CGkXNf1B3r^|&%4$kI4rY^oPQEU3>w755bU&x*C@iBW8$Y0k4TMVd&Kv7tE zNZK?h8m1x$aovqQut$IwviSLp6A#*>3&!VSW2(!Gt-=a1tXVs4_70@rM9TWYycD4@!^pcJQbKh7Z#91z#mw}d`O zCjcsT6FUTR5J5QTbE3wjVsKl zTD2VS?%f!{Y~FkL7-Uu|gDZ3Q>1RfAHwAxUQkQ!gtQ)QteUv8bQ;bq^NRODpwDuAw zwNsYJ(`R?8Oh>NOoju;{=`I2HwzfOe?nE=J>k#&FY3z z_8W~?uRY7Kf?a;&bGol@j;d^U<9lBvi3<1TfejASZ=MDl`%ih@ySMso@X4$k7ml*( z%Er0CvZ2O6$7+H@oT}n?n}09%Vv)O?rhT~QxZP{tOjEBm&4){imJM5SYsClr#yVVN zruB~o(7)-R^c+v#rY&xN#!#GV@_{P2*Sn*Qv;&#u{6kk|cKzowF2@C|NBc^FL7r|&ojSHv) z!x55?NERzW_6Zw^C4&?yL0WB>Gv+cs`p1+3)Jz(3Mglz&IXl@q*x1;}o=v(;Z$ZuR zhnaI-NwyfEVyNtDS!jT07!#flW*Y?U`vc%7!y@FUrBG@SW(69FRfrrQurtJBy07*U z31CKbZAf2;qA-o9|!{*VY z1wYJ)qe#IW11wHYcnznK;{es?4eab*Rri*gjnC535Hpo?Z>8 zKgX0zW^Pgkj5r2Q7uM2yvM@FxKI=0~EO?T2yUc_6#&CEKG%k^xu3p_vnle%7K$M19 zz~F$y0}T0OkZ}Xv$>Zkq}#<|^!WrVMgOlq}2pD6#Llz2frPBX!*|4qqNE9$<@FlRG4> zCpEuD-zK3bs-~{Ers586^zN`9J@pbXAF@TCiC(#3q1bp#Q)^`3h1RWA%?ZL5x}2_( z>XcVONj+DOu*dzVI4G;5`gu?vEld$h^GDx_T{)k{Dq9nN>AtXHaLn<+?bU48-iyg9 zExH7XXZk$c=wiUC7;C&?filcdSW}L_w71~Q+mV)p&5~DqpC0-ytaka@yiZ-U(ve$J zF9$Qa8#HqcxvN*K81D{>upbJ3KcmAH9eB`CcVh-W0KP0VvXhoBC_QaAFH=5&8%#E!I+neIzLp3R{#N@i zAGx{zuhK5)oO0h?&8G%6`ULngl<2w9HmaF6+$g=>k6d@sZbXnuGX22csSvMr7!d!8 zJUn%Xe1z1V5L6T(MkFYghQWloH8^PZD@M|iveh}S5P`-)8&p)uVCew@FFD{x!W@u+ zq6or^@UeJcPDAkDQbV6mID-S8j4dFex*rR-rgt}xd4Nom7z9w?d?tr^-2sYDQ~K)# z()o$k;4fy&gOW;gT)6?DWSq{l+FL6~)@A(cyKxf$YQn&6*HYcbbB+vp5Xig+xjMsE zQ&B+!dqT~EA0AA2m#HAW$EIB%Hk>5|Z8?xCNh8DFfm680V6>ZI6HDfPV-E0U6wkhK zxEC$)!xK~YNcq7;g3(gJxDD^p&A!iWXV~oT;(=(6K>N)W-z;|mb_F!wtG?n5`I}!- z7-rHkp4Xp3oLG)zGu-5`paVm5mhO2DsR>4F;?W(&PG6g1q>PtDQY`_DW|kl3%E1g) z4q-4*26K524$5musqW@)W}E5Ifc8ek!M2f2T!QxZzMgjiJ9o9EIv((C$0dC6Q=3n> z%y!9Owrfp+-3@^hE+dM!jONl9y@TMBP$mCkR2+A*{J+QSy?lPg$9rH$Q}2$vrgN5& zk0)4G$uXQ*Sf^P$GcYLq;Unkm0}=B{YP3J<9-u$mM{6p5=%AyNw3g|C0f(HXiWOEI zO>(CbSY6Eyo~a#E$(Uq5Q#;Q5IgC0{@Igkj4JXsk`N8cPdP+|wqMZB=z1Vf-PN4AV zSs%aNSIge%3|PEUq7TQ5u{EZPDvMl*8fLHmXtUh=xaPFO$B(xgkFrM=^|f8+n-zZb zxFN65Y-;?RkIK0fjv>~Sq0d?9BD%*qvxVEc3_~xy6TZ;4rK-H`!XdGixYK@)D@`+x zcpEo|oIdw*;k}NS@9px8XF`nL2T5@(_*+6X0%fU4i-&cApc7!%fwzxLPVPYxS72bE z)yot}@HL$$FB5|`MDVo7NRzPZq9|!G2c?6DKQ#@FDEw4lY|#^N?zza^i9msPl_Vl; z2c)weU>_7$BDhH5U12m|NV*7%1Qoc>GoX_3hX9k}A}s~rPO60
2-xrg52{2)8% zBMyM`e?vGn0{`qfqrntFRN^?w%<)9Y)U^^}nHgGNo8#;VlA>8U0k?qyYRHIJ3z4Py z$f^g98zPo(-pfbOMhw@Gi!i`HLiD8wsYCFe&G20N9r}+3?QqF-#cAaRBkro=;h@Ia zzwb5TKY#!N5ftRWCub|J(#_MEJtS z(u|n4{VZx~swW>th#kE9IopDk-PQH!&hs_`{r%hS&Stc}|0U~b+hwf=&kb)S`ku)Nxgd9==einC6P>}xuFa=Bz_)JW1WRyxIax2xbq>t2tHi8v;I2~Z(=hgnR!&CE0JXGV!puO-;y1I z9UGwnB;W)YPj_)S=R}`)>4~5ctM5hE5M&2BnNj{E$EjfjO(x8WLp|L!&Gm5URaRD3 zxHvCh8o)WY=`XZq1_5kD-vpJJ7=@cJVHM2^~_1{21$v}+UOHV+P* z9IgDo?UZ@Y-(C3S4$~%!@4wFKeDbMlo{}{=s?E@T?tbeghC*R6pNw#)#ku(S28Te= zqwJlv6e9+|Z}8YE^SIU8FXp!veh@GBnB(gmeS59$0- zOEB|#ztY=R(pyOBj%(bt&vMPASkNt5MA2=k9pjb7pUUcPsNVQ<#emA6MX@f`M4h&v z9fmLipbn%_f`nuMkcN;9R^Qsez=VHk0dN<5a%TSy>HNcR<)Kj9~IA2?Ck8!F>H|Jay-<5R0EVICLRn=a>z1-0Jb`90V5;Um(rk%62629 zLe>bbm(xH~2^0>xfs}HoaJ_UA=vGivJd`m=h3El-mJz*Ov)3T{>(T3pdt+?R^$SRx z4#Nd#AmLK(UqTXEA&Zrhdm`rVhX*LB4*th zu-(lMnm~r zZ+9&T{(HgtTF#*mXRkuLGBqx*de_#<#hDFMTQlgWvm{1-e|Y+Wa;;dczRO_mv6}9F z?G2p<5d$rHTjwSvzgqsf*4;4rUTyV6Qm3GHtNU_=_r^nWA0!=Zx4KO4y62T6kq4U+ z&VwZ53ckIrwzrmIwN}O!y_TNjeQ7LNTzyvtN6{gn-A|uucm2-YmS0eIB%ollh{z4KzKb5=G16_duc_Toik`QCu7wr(vAXOGEQB=VGMzB6v!6y#B zx&(L&PizZm1|V{D4BvdRK5!O4!3iPK_NeioJjhnkHH-M+(y}rFM3P+&S8wK&>icVJ zTo-5d0eT>A9Jt>%Y}#a5mSpeXV2R6UD%;~w)Om=p8psW=Rj&+m!hnV6@zxQm=qc7$ zXT38%5BzC`#QRJ|kahx~1C%G33D{MLN14FTzv}Bh-CIYZXV6G3h>R&n<~KvJ40MTf zZS+naI)HD`1%HhJW(kze`GQ)gjaL^&c(uZ}&@$ zi`%@`0za*Kl7~mJSz%knoyfX^{T;()Id6LIBKc5R%8<)F^P{IFlwSB7(SOk$qO%;7Gu_`W6*5QCFy>E}#ciwX}yV~$jfKLF`U>?(D9AMl$+^9$eb=_5y#l)?F(BR z&sL-=?(3cS`ewxo2G*Zet=MipF|e|d`N$ps5V1a!EG81Y2w8PLt_ds;gn>mI*IEs_ z6bLwp90>6!#3%v);ur`6gbop?4>nUTFwImK0Y8G`{e*434++XR(}>y(a`36KO03%5 zxYx+3hs%e^fN(w8=I>EO60Zw3+bV$?aVb3HvBz{naI&JEB-8^sBZnS|4#3%o`OK!w z#0J8P&k%+afFErJfE#*oC+-7>gp2Py&+ssiG-!(qpi?3<2=013oL`vD)DTDk8Ye5z z>_b`Oa<~O=swM$Mcm#ye0@iXcQ@=gmcC8m~Hg6SbO*J6~6?)*c#O&^bhc{#P+TFRZg+m>o~Cq-fj+bOAh&n2JfzJ8%R^MEN+S zscZnk@OeN7d=HTV0m&hD8bbC%ZXkzu7Mp83UU_D;Wi`pkCYczRDw)mWtN*|oNaftH zd2=CBGZUO?HhT-0nNTPEUblXI-JCpM*uNl$f)4XnQ9~{JIV@91vI}^1ientEn2)*5 zxrLb(zc;Y{tg4+%2ss#n8lm+hi-RbRQMvr^$Y%U>8P#z-$_dd@=|hKGGt;8M5oa9u zX0VxVa(j_F;-TfDV*b%{Wocsgjzm$u#Kc*9^S_u^Z=KWU4|Ni4xUNv?+L?O~ptf zo+lW{%+~&)-GPPou%N5v%7oMWvdRpv=y=6x&8gfOYWM$`1|Pq7a;vi3Lh+;&3d8>O zD3o&9FUN8}ovkq$&!?T` zZWvr_EYR7ov9Bmw*nMQ$#M@3~ty$GW?bzRnM+!annRy*B&)r$k8N7f2A1n3{qx51u#j7}T`uSb=ff{zX{MJ5t1g z=U1@v??&n=oce4eUJMgaR-f)ePXS#fPFmnDWQsrxhGK$-D9J}ba`DaYE>PI*#4e6B zpHMJN!C=x85Vcu%UUO-Q^ zJ_OBJ*MTcbLIi#Dq%oKl7C8F$1Z@itNNBy+cfoI)toS&Vh0N@AcskwVhWAV@xWDk$f*F&^h6`vwV2Oxhn#RrKE^Q~*f@ z#M)zh8xDlhN@_aZU62;Vl0MqaDlZt#?DXA=d{TwObV*ZobRBNN^KDlmG7bxE zxOL@$s*E~*cbO=V?{ANON7>eHUp~hS)sH;tfaskhr7g)Sd+KhqlgZWh;C;z(AE}>-vV@_6oK1 zR+|H-XUE2}_3qP+_8vROL&G!_>R{;leNeV5MPGQNg*|4dF=a&^MaENzHR|SlCTrc> z%pdF$=PW#GcMH_yzq8pAy_MBm+M-#Lr|-Qh{mJeS;x58KHiBNCuWv6Ooa#af#||WP zfS-gMmjMrdJLo`TU=JlXG70iZYq8l5m=nw&9SXlO;1GAZq-4TW9ek-I!`o)g zu!@n6HPVx|17%A9sFM(o1UBM&Qz|!5quqan&R|p^x^XP|;aH?GtI^#eZKvR?iG3B{ z&>k-SW`SFHZXg0pmf*4Y&U^UyzDQ&WAtE2Io@CbmH6x{;fFM96@6ie%GK(QyELsmR zK$wZ45s0LPFHN~F*-Bgdp}_dpzHP$3L#7}wCzz~8fl65+K}GZCFQ&|p(jGOqB!HlG zCzY_#v_qE|hV6vl8w5z9A~zl`$y%J4VYDfw7q3$6$kc?CpxCQ8T`CRXIPIAh5bKAmIsB0y_W1Mk8VrP znO!fNxyf6n|3%H>o4RT44=ic-4D^5PNPQ!0t4z17z3Hfe|IB%6=T}7|hV?(c#crs~ zyXSbe#^R|thnJh@@6TcS(!CPOcbK-H78r|Nn0FBo(3hI^K*_VzS578Iq#n+x6?5~K^+O#&V zMpln{^~C7aChtn>5HW%Je80o1>|-yB=GY~yoKm9GZWdm(O!>CMxB69mHfCET2i1ba z%2$jRhe`uUiXmz_O(mfRtD=Pbay5yKLRX~QFmV3<%3ksP_K znk}fw!{p!i!H|LKuawl@nvfby-Q>dqoI?X9MDjHTY~pGss#t8(q<Xb5Ihb(>~gX`FDpx8EyMAbA}x!!V@Cm?`@DQ<^Cd}3j`bAJUnJzr(S6u>5OqX# z3#HCSmuBb@nYIrWr2QwjgizsFgNHP7ul{)~S9DNiwmjgje^S+(_mYzL(xBw!z>Y&T zE@K9(QZ<}|CavF_jLD_?4qkF>WNEM*Y#5ZQPS+oK7&vTgf#|5Qu`$>)ERHf_H5qD2 zXgW`9t8IMH>;s5@>hEtaTfU4~0m#$m@7y$;*bsKfAy0xQZT6H7_yOQ7ftqVy%$uM< zixNE&Q*$xbBBjF3)igI01HJ;+WbRRSG&PtXkwq#B|BOdL<)WzD6Wi0HRn?o-zU?o2 zEbl_~A%?a1VVjQ%J0bCc&TVuNj8UUD-*a)hGIxHxm_oGt?1Zz0Yopce4O;YS^>ntw z%85?A7EM0(k(!*XChEb_I}VNgl2ds9>+ZDuvuF0R{1IXms}w)AS=WIbI<`1xl>b2E z)(-ZA{uxiGL*^bdg}rlN=4x{YbbldbKN4v^$MesZ%i3R|Xv663ww-K4f_P}XLI14N~yV0jX zU@3sLra3eEh);M7%Lf)KD3Q{KV-&zE4&}_5Vx8^{o^*helA_)@`5Y>tH3(q>4GGsX z7=ZU!aXoKD5ycG@p2P(X0yA;{lkqweT%veQ=?16UP(BWG1mmGYiJngu&i&kP$w&T1 zESOS+yTcyGmTKBk4uGh6sx}xhJiw?Voo~ofXfggnv(f15adYRd_S_nD0SuY_dD^c; zWcSf|KIqrh(diTn^joR6FL{UiKAj#8>8fx!T5-}=%dtUG-n{u&Lh?T`i;6k!zahi3&7Fwqwv zclx~1-4nm3j@&W&t|BKVxAD|&*Qmb+j}aoj z)|EA?w3_dIA8D*g+T$$hZPMlbxwh=V4lR1j<1~Rx)9qJ3m8D;)fW0mwJ8q>^lTy3S z{xLZ@^NYON9;jnbZPQC|6ICQ*I9=!s4cJICzc6WeddjW5=w8ky?bs#-4KNPO< z^e-aq6gGbct!7O(iowUxGDHlvp;!z7s0l z?C6F=X-_zqVhKeD)E=c*2I#l+`x>DWu2PfT^IxIY$9vYk<1 zIFu;aJ{H|;xo_yran zqG##JoE^99+8=F-uFIJp42V_T3cji@OgA_1$U}{_5VJwEO%fI$Xd}rrpdN_``yD~dfmD8<{&3M(o+o#zsa#iMv}ti6K}T&DJ>>+iNSvyh=|wMTs{ zTO@%-oCX?EXVwRMhXPlGexvBi;F6>X@l>|*tF2}w>9HqxsiqJ(FxKYN_`>gAi{V&L zvF>hzAJ|szJ8Ze#XeAzyXtTS^{FSk&9}YL4jH%-JDW%MCCtf(D?l%QLs=a%MGh-rf zis|UE+%AEd$>Z(c;=Wd`;-9uQ+vaQO{WYe6KC1Vu8DR2!SuEW5h5LpYPT|Q`vz>q~k(jojTE-1JIt7-@|jhhjg z0Dc=O_rNuvI>>_>R`ugePAt?|Ouj%Qm4luQ>H))7dof0XN76}B5kc+_2@r5Qe;mmI zBsCv@87UJRGs%1|f*BqH-kpaJr{sNXLHG#??|~`13pJ^*d6Cf#EH~4Zxt?$G-3qKn zT>_yIc!!QFWYfYC%ZIH*5Q0@Q;g{_T;ngjsC1I#QsxKgR1;M{6@*dnMO3(8*>Z}yt zvy!c3;Y|(_NtScbJ7S@4F;?-dUR7&f)86r$0sf70%3{IB$_7IztWT#VCf-04NYb3Z zjR7b=L}YWodZ>t{fyf5HJ#yfd!keFw7BnOJ`&_wbNVek?*~bu#O??dSDI96tkodqw zJi9HmyiOj$l-TS@xR+4CV>mHLJ}TbBi-b@CCUU{qeRloi_9(;CWCf>T8nkd_|7Fw@ zT)uN?>ucMW^co}%RaQ$cD z^4^0-ztxmK{odn0R-nDt)sEop-`vQ%m?*=^9cW^(+Evd{MWt0y=(K6~=4 zKPS)`yTDTMZsXSOq5=BCO{|?&ZqJ*?ixv5rr=ATYJDgd|scIuRySO2|p5^Az))-Ar zwuJ2#surltNyMg$F^Sjlx3TSQ_{bg-#R#DD;j9vClr{hQWegG0U626qnneKL8XAN{ zb`^>jHlb9V?9*Q(JRPNjNEiar(XnxO)b@9i^%EsfLndUi#%D#CO>Bdt69lyYu#ORf zDW7g}JB3mrMK4i!hyI89!(J+??Zwh3k6)5hxyrh};5CwZh+z(5DKLj8umO_pF#BPz z51`PYHMxuY1nd$+dmcgpY6CsF{TI@2O9X5rav9J5wQ)@V&Pb#eNv}|cv;PuaS2}~KQIX(z&L>LflB=LmWtXHt5#)BO-@af=p+TETuZ-ye?FPnhNhM- zvO|roc1?o)WrJqQ#fode0T22$Icl*le{>O&Gv%wQJ%3&EJZ7xU0{WwGa3ey?GlnD| zD@YY+NYf26qN|Ca2-+;%dzP+|OLLTl3T}!Ml20q}fAlS8;EW*(7n~V*B3M6G>-w3b z7Vk8YzlvnlApv^R)~#jOk}$M#{`}m%`|s2FuSJ;94~(AQ^EZ3P8X-*@iVAzvySxP|rtP3?`_`M3TK|988;9+%xul1&&+ew>2C~>&jwyhdy(UO(axRAMK zyS;0XCiEms*7M3uKFo8IIi5dkdoOs5Nwh753ysXjnZz}EJMFog&gCW3To_5y&(xea znEu%KXrP5OVD8N?kpwhOOj@sx(~$L&HjFRK2mzvq}!ja-d9n} zz&Xe`K%@3$V#}D!KA~E-s{Xv8tdG0KC_imUt8rWIQIZM(JK@oR!va`)v4K=dOf?sP z>3r8%E%C%ErJPI_7&4;}qel>+p%UAn<{7U!jZDlfB*hpKtP2Q41g7MI*W?;*bt(Ou z0Uk+=J@bll>#K+ou0Q_!mJGgfb`OS4wU#r}9(N`dotPLx-ad&slfP~{O0Vfq|Bbk9 zUzX7tt1i10C%$iBDw6+Z5u1$6P^{Y6!=DzM4no{#jHTO*TpryoO(C{ILfw+WF47bV zkv?$PT_l4U?=o@W`CcA^Q8ETrAB;p8v0xovu8H9XCGdB^n_NyP${;$k=~>MmbAvrB zyT;5qZyq6Y8wP2ZQac0Zh|7=o5&%yGrwn48Cmn@QerXO|^!!%z!ACZ@5*?rmhtZ#B4cj784D>HVEAUYGYAuBe_qJU4BkwT-Owajf~_ zX)i3>81iZhI$d~f`8|`r)+_3_*tOG^V=5r1jL5=DYE3RZ9P6svoAi9mWd3r=S^cA0 z!{$}AnSA9T$^nI&*!u3lfHOcTC|rO;#lxqm!ec}seEZ1M*wwP$@c9Yl1DyUsPa-L! z7E#+C-ETU-#JDATkf{`RxL=y2+g*kXI~Ce($_LBNKQD*uuEup zP}Onma*aA))p^fd-xAHAq2SpW;!|y80h9h55`nZZy~|7d1V&1n@nMK~!Nft-I5+AO zs)z#Bb#a`1D7}l0oaR8X@kx-fkcokMdPvKuPDn|$Ja~mDTT&!LdSURfUT=JlKs`=! zz7v5k`B))Z55~=nNeOf&nORs+PKPcS(px&UB@Fs*9cVp>*oX)$@VDdcqdx{cDscsq z#(K;hz#=cAkc+6Kz%`IG0RpJwhk1Tb`3crM1$bkeOu$-&qh3-{6nuOC$^p{aY;gr^aF$zAZ>Le#2uf&l2!k3fz||2 zXEnxt*YB{M76VEltR3!6UG2N-*Lc3kSq_>Y4H2Xsxvl}ep_okvQNho-RPC6Y%j=@X z)uw+5t&*b&WMqh{dQdCKCy{5EZu*t-jo+VQedB?Qr~OZ@b?!7=HZ-0Ew~O?zW~860 zx0~>6=ys;f@!i@QWTeAKd)j$R%?EZm{twQ2!8r$CwfzoiZ}B)l z!+WjY^u$7Z<)8z@zW0N~c^2vtESR{_rj=rJF=lzP%aO`26XIxt9?O55omHQ}Uw3ZR zyX#%@vo%Y1`j?b_){pLM*x-8GKgg)VBxPjrp~U=@@2;$`yWKN#*|Y~kI3GF09XOt& zB&m|&n-n|sW!PL5ANS@ z$Gb~RdE~OgJ1Ds@p@?`t(mYKHpP^iXOSBeo5I~z8TJ=oeFieN!Z5m-QXTUBH0XV2D zBt-9lG@PV0lNuA4Gl?M!)KNGnG^FO=A>=mtrMr-yK}3(F5doJhAuX`)>Elrd{DBHa z${Ntk51|4AaX&<1$L#N}O;R>dyhgqu=s;37MllJ2*KukJJB=PjW%1d*a9F~wD6kkD z(P{y@VUMOlJN!d#P?B8&#$OvL&`hjUn7Xcs95mu{v}&?i=Ubv33IG!<~ky@Ty!+&gBj>ESFt^bwOW-uJ}Cxp zd@mYjAZMD=#kcH&bYP`Mh-$@)gT7|!5cB$@BV=w)YtoWR({;fbi~lX)<1oDtVnQxx z)@!a3lCe4`CYi{Hh$O66=s5Vo@#tLjo4;^jGKnewA*APZm(Qt5SESN>o(z6(r_Qay zy%I`V@WL|!d;@X1xLT7eT(@au*$Loz?o0>=gj7~psr6G6s}1cYdql@S`ARni?Y+aeP*lZt{6hk(#i>76 zEFL(3^GARGmZbc~Cge1gVghS{wsyi)2R;@OyF9f?9FWa-`(Xpib4am)SD_sP1Tp^L zIw2*Blamop@{!adJW@7p*pNqp9icTR{Y=oMN&sQURlH%_wx5dy7>qs=I0RovXD9_{ zxdR!pag?Di;j3C(TftZXze8=94jg<;b#jEQCS8m0^?``clxLyDHX%6;r587A9+@Id zjktv>Dh2=yJ_9fqH!?CZZWXrwew=^<8cpAqg!(GhHO#6KGah^{KTLnIDJbv<}s&z5v{!~A0W#)|Zb?zIe zGt6|Nq$po0feYas$I?%(&M2Fgj7c~*Ro3rP?f7<9)V^(8@(}LmPS};XG_#mPYD#cM zw6%3e3-@RJmPjt(Fb?obeXOVMmvA;J3zdi}-g5SnC0YR+`=8o2^3c-K*5;XPP5;F6 zi;m%w#5iAlxmf7M)m|mF{k@-18FeCb!-km8R{ifenBETDJrOF$+AlFFxgkS);>4cq zw>mp@NL-e+^#(#L1zh2<*d1Yl9UPKug1n8+k&_r|b;>A2HEM%yxs;WY&>NC{A;7!`HWeb3p*2Y6DbfoZv zGL!N>EG|Bt5VcM)*ENF!10#E--u9-d>Q+)mhMwH&pf|z{$cT*(4efa7%rnoRprmAT zCCPS*`IN&a;FyglhAAp4QZ`<<@(!7$z~fE45`J=jLNKxZJQx^&3RI;25u?i}{31qq z!;UhPlwtot#z;K|I&~GcbrmLv3(?UI-Q?d^#0m8T%u*G3LhCiO^DMRR#%nIj<1+Zv z!P{Zp&@YqwVO$*g*wj=Z%CNe+`Z5r}gD<8sG!zvXh+BtK)?()7*{=zu)Ko)ARpS>#m& z44c}G`*-dvXp5#L)k>>fBxgcv7J+FlYe*xgKC;AYO~d2yi6FEKJ-y^x-6q|#Ru`zc zU=YaT@eBk4pd06KR{EA-cE=W;Fb;qq>FVaT$CO~PSmcNrbesd!J)_PT^~^-gBU6LJ z!u(;-Qc+nM`mt#p*VfkNKrj=E?2x7NrWu+Go(3&x%q23#PL>{EVK7@%K=`1m+b$Q9 z5>NMD7?`N{eMH7a3&VyAkWI8Bt6^U7rYmi2Z2A}Z&Q4DKc?U~L_-AxFf%9T)OjT2J z4it0Z+9l)+)^Ww_b;(3}4}@MIUF1N0=}>3%6hOoUQ;Q_v(NbCUXXHR75S(bCJQ)Pz zyH}hyG`IOSrbLD^#MWl1?D z%jd4mn4+Wxrl~i=pPex!rK|Lmx_D}{ukwJ*C4(cCUk|1w3_Go}#V#4D;XUjtQ&Y3; zQ~pk1^04YWJo`>qdi1_Q5$=H9@ptO)E>SQzjE|L#||@r5iHhzeKq;=Yx%eR ztxA}-qB#T6gwo3!RcW@Xy$`Vab!d}Z)I zwFZx#fOir*0pk%EZ6MN+wzjsExCjFI@cDek$Khei(9qD^O-+7)*@uHxIUbMWtRS2_ zVyq4fd=F7S>s%&mL>vwm-93$9dBLkw&kiDZIY7ofyqCI z+N>Y+BG@WN#LH@H3+dc{1FHApDB}oYl}jF=PX* z#u20mVLlCMXCq<$N&$3Rj>(tgLc$(pv7UjB6mt85=UiF&$BfE6qW~CcAgvT7fbZ1Z{XD!66|?L^Pb; zUIdd9#VhvSuq5ao2Zx6GLDkM^w{7u{noVma!N&~s1r}&%LO_E9@knjR0MKl{C+$HB z8lRjj1112xArvec8?WQ>GTHq;8M!KR3&;4zTyd@hx2QlH5al8b#EiIoc@;-L4q2R9 zw#t7_vyKnMGioY^KXC|wFX~P!4<}b|mYdw%G+~ahd{@t1+SkYO9@_6w{Jx(cNY!0tX!QeN>fy?>&%Fk?%UkZ zf*jnbbR16%nzj#h79TA0Q*Du4-RLv=;r5{NNP_IHHs4CfXt!#R9I=^O=HJ$Ktuts| zaVLOP64TvPCtRZ+DEgJ-nYHzU8lU4fLBSY6VCe0rWP75X%v)ZJt4af^3C>Y7Zbn z-yR zXn)m{(h;jPf|O#QZ4pmBD0U4Q_d(GJNg*LA97*+)$v2>wM1aB%Fw+HWM4TrjW9pRG z{kV2(N2rGtk00+(l9!gwrE{XkjoA{H?jU7yyii#IC^?#n2IOrzq2D#4rR;1BMx64O%f2 z2q=(ZARG0J%NWnW07h9#l$`npLPsw@-T`8kK9@tv*xR-LjlB|s+Ujf)BlX~dFEs8E zb&KF->*zS8cK%P$e5r})FDpilxe4BuqBi?f4TnOf+XRI1W literal 0 HcmV?d00001 diff --git a/docs/kafka/import.md b/docs/kafka/import.md new file mode 100644 index 00000000..a00f8c88 --- /dev/null +++ b/docs/kafka/import.md @@ -0,0 +1,148 @@ +# Kafka Topic Import + +In this guide you can find instructions to import Avro data from Apache Kafka +clusters. + +## Table of contents + +- [Prerequisites](#prerequisites) +- [Import from Kafka](#import-from-kafka-cluster) + - [Usage example](#usage) +- [Secure connection to Kafka](#secure-connection-to-kafka-cluster) + - [Usage example with SSL enabled](#usage-with-ssl-enabled) + +## Prerequisites + +Before start, you should deploy the latest jar file to the Exasol BucketFS +bucket and create UDF scripts for importing Avro data from Kafka cluster. +Please follow the instructions from [deployment guide](docs/deployment_guide.md) +if you have not done so. + +Additionally, you can read the [user guide](docs/user_guide.md) in order to get +familiar with cloud-storage-etl-udfs in general. + +In this guide, we will be using `RETAIL` schema and `SALES_POSITIONS` table to +import data into. However, we have to change previous `SALES_POSITIONS` table so +that it includes two additional columns that will help us to manage Kafka topic +partition offset in Exasol table. + +Please change (or create) the Exasol table as below: + +```sql +OPEN SCHEMA RETAIL; + +CREATE OR REPLACE TABLE SALES_POSITIONS ( + -- Required for Kafka import UDF + KAFKA_PARTITION DECIMAL(18, 0), + KAFKA_OFFSET DECIMAL(36, 0), + + -- These columns match the Kafka topic schema + SALES_ID INTEGER, + POSITION_ID SMALLINT, + ARTICLE_ID SMALLINT, + AMOUNT SMALLINT, + PRICE DECIMAL(9,2), + VOUCHER_ID SMALLINT, + CANCELED BOOLEAN +); +``` + +Please make sure that the Exasol table column names and types (except +`kafka_partition` and `kafka_offset`) match the Kafka topic schema. + +## Import from Kafka cluster + +There are two property values that are required in order to access the Kafka +cluster when importing data from Kafka topics using cloud-storage-etl-udfs. + +- ``BOOTSTRAP_SERVERS`` +- ``SCHEMA_REGISTRY_URL`` +- ``TOPICS`` +- ``TABLE_NAME`` + +The **BOOTSTRAP_SERVERS** is a comma separated list of host port pairs used to +establish initial connection to the Kafka cluster. The UDF consumer client will +contact all servers in Kafka cluster, irrespective of servers specified with +this parameter. This list only defines initial hosts used to discover full list +of Kafka servers. + +The **SCHEMA_REGISTRY_URL** is an URL to the [Schema Registry][schema-registry] +server. It provides a serving layer for the storing, managing and retrieving +Avro schemas. + +The **TOPICS** is the name of the Kafka topic we want to import Avro data from. +Please note that even though it is in plural form, curently only single topic +data imports are supported. + +The **TABLE_NAME** is the Exasol table name that we are going to import Kafka +topic data. + +For more information on Kafka import parameters, please refer to the [Kafka +import properties in the user +guide](docs/user_guide.md#kafka-import-properties). + +### Usage + +```sql +IMPORT INTO RETAIL.SALES_POSITIONS +FROM SCRIPT ETL.KAFKA_PATH WITH + BOOTSTRAP_SERVERS = 'kafka01.internal:9092,kafka02.internal:9093,kafka03.internal:9094' + SCHEMA_REGISTRY_URL = 'http://schema-registry.internal:8081' + TOPICS = 'SALES-POSITIONS' + TABLE_NAME = 'RETAIL.KAFKA_TOPIC_SALES_POSITIONS' + GROUP_ID = 'exasol-kafka-udf-consumers'; +``` + +## Secure connection to Kafka cluster + +Since the recent releases, Apache Kafka supports authentication of connections +to Kafka brokers from clients (producers and consumers) using either SSL or +SASL. Currently, cloud-storage-etl-udfs Kafka UDF only supports **SSL**. + +In order to use the secure connections to Kafka cluster from the UDF, you need +to upload the consumer truststore and keystore JKS files to Exasol BucketFS +bucket so that we can access them when running the Kafka import UDF. + +Upload the consumer JKS files: + +```bash +# Upload consumer client truststore JKS file +curl -X PUT -T certs/kafka.consumer.truststore.jks \ + http://w:write-password@exasol.datanode.domain.com:2580//kafka.consumer.truststore.jks + +# Upload consumer client keystore JKS file +curl -X PUT -T certs/kafka.consumer.keystore.jks \ + http://w:write-password@exasol.datanode.domain.com:2580//kafka.consumer.keystore.jks +``` + +Please check out the Apache Kafka documentation on [security][kafka-security] +and [Kafka client configurations][kafka-secure-clients] for more information. + +Additionally, we have provide extra parameters to the UDF in order to enable +secure connection to Kafka cluster. Please check out the [Kafka import +properties in the user guide](docs/user_guide.md#kafka-import-properties) for +secure property descriptions. + +### Usage with SSL enabled + +```sql +IMPORT INTO RETAIL.SALES_POSITIONS +FROM SCRIPT ETL.KAFKA_PATH WITH + BOOTSTRAP_SERVERS = 'kafka01.internal:9092,kafka02.internal:9093,kafka03.internal:9094' + SCHEMA_REGISTRY_URL = 'http://schema-registry.internal:8081' + TOPICS = 'SALES-POSITIONS' + TABLE_NAME = 'RETAIL.KAFKA_TOPIC_SALES_POSITIONS' + GROUP_ID = 'exasol-kafka-udf-consumers'; + -- Secure connection properties + SSL_ENABLED = 'true' + SECURITY_PROTOCOL = 'SSL' + SSL_KEY_PASSWORD = '' + SSL_TRUSTSTORE_LOCATION = '/buckets/bfsdefault//kafka.consumer.truststore.jks' + SSL_TRUSTSTORE_PASSWORD = '' + SSL_KEYSTORE_LOCATION = '/buckets/bfsdefault//kafka.consumer.keystore.jks' + SSL_KEYSTORE_PASSWORD = ''; +``` + +[schema-registry]: https://docs.confluent.io/current/schema-registry/index.html +[kafka-security]: https://kafka.apache.org/documentation/#security +[kafka-secure-clients]: https://kafka.apache.org/documentation/#security_configclients diff --git a/docs/overview.md b/docs/overview.md deleted file mode 100644 index b52eec69..00000000 --- a/docs/overview.md +++ /dev/null @@ -1,193 +0,0 @@ -# Cloud Storage ETL UDFs - -Here we provide the instructions to setup or configure each of supported cloud -storage systems in order to transfer the data stored in different formats. - -## Table of contents - -- [Prerequisite](#prerequisite) -- [Amazon S3](#amazon-s3) -- [Google Cloud Storage](#google-cloud-storage) -- [Azure Blob Storage](#azure-blob-storage) -- [Azure Data Lake (Gen1) Storage](#azure-blob-storage) - -## Prerequisite - -- Deploy the latest jar into Exasol BucketFS bucket -- Setup the ETL user defined function scripts -- Make sure an example Exasol table is available - -## Amazon S3 - -In order to access the Amazon S3 bucket you should provide the following -parameters to the cloud storage etl udfs scripts. - -- `BUCKET_PATH`, that starts with ``s3a`` -- `S3_ACCESS_KEY` -- `S3_SECRET_KEY` -- `S3_ENDPOINT`, for example, ``s3.eu-central-1.amazonaws.com`` - -Please follow the [Amazon credentials management best -practices](https://docs.aws.amazon.com/general/latest/gr/aws-sec-cred-types.html). - -### Import from S3 - -```sql -IMPORT INTO SALES_POSITIONS -FROM SCRIPT ETL.IMPORT_PATH WITH - BUCKET_PATH = 's3a://bucket-path/parquet/retail/sales_positions/*' - DATA_FORMAT = 'PARQUET' - S3_ACCESS_KEY = 'MY_AWS_ACCESS_KEY' - S3_SECRET_KEY = 'MY_AWS_SECRET_KEY' - S3_ENDPOINT = 's3.MY_REGION.amazonaws.com' - PARALLELISM = 'nproc()'; -``` - -### Export to S3 - -```sql -EXPORT SALES_POSITIONS -INTO SCRIPT ETL.EXPORT_PATH WITH - BUCKET_PATH = 's3a://bucket-path/parquet/retail/sales_positions/' - S3_ACCESS_KEY = 'MY_AWS_ACCESS_KEY' - S3_SECRET_KEY = 'MY_AWS_SECRET_KEY' - S3_ENDPOINT = 's3.MY_REGION.amazonaws.com' - PARALLELISM = 'iproc(), floor(random()*4)'; -``` - -## Google Cloud Storage - -In order to read data from Google Cloud Storage, you need to provide a service -account key file. This should be uploaded to a secure Exasol bucket in advance. - -**Please make sure that the bucket is secure and only readable by user**. - -For example, - -```bash -curl \ - -X PUT \ - -T path/to/project-id-service-keyfile.json \ - http://w:MY-PASSWORD@EXA-NODE-ID:2580/bucket1/project-id-service-keyfile.json -``` - -**Summary of parameters:** - -- `BUCKET_PATH`, that starts with ``gs`` -- `GCS_PROJECT_ID` -- `GCS_KEYFILE_PATH`, path pointing to the project keyfile in Exasol bucketfs - -### Import from GCS - -```sql -IMPORT INTO SALES_POSITIONS -FROM SCRIPT ETL.IMPORT_PATH WITH - BUCKET_PATH = 'gs://google-storage-path/avro/retail/sales_positions/*' - DATA_FORMAT = 'AVRO' - GCS_PROJECT_ID = 'MY_GCS_PORJECT_ID' - GCS_KEYFILE_PATH = 'MY_BUCKETFS_PATH/project-id-service-keyfile.json' - PARALLELISM = 'nproc()'; -``` - -### Export to GCS - -```sql -EXPORT SALES_POSITIONS -INTO SCRIPT ETL.EXPORT_PATH WITH - BUCKET_PATH = 'gs://google-storage-path/parquet/retail/sales_positions/' - GCS_PROJECT_ID = 'MY_GCS_PORJECT_ID' - GCS_KEYFILE_PATH = 'MY_BUCKETFS_PATH/project-id-service-keyfile.json' - PARALLELISM = 'iproc()'; -``` - -## Azure Blob Storage - -The following parameter values are required to access the Azure Blob Storage. - -- `BUCKET_PATH`, that starts with ``wasb`` or ``wasbs`` -- `CONTAINER` the blob storage container name -- `AZURE_ACCOUNT_NAME` the Azure storage account name -- `AZURE_SECRET_KEY` - -### Import from Blob Storage - -```sql -IMPORT INTO SALES_POSITIONS -FROM SCRIPT ETL.IMPORT_PATH WITH - BUCKET_PATH = 'wasbs://CONTAINER@AZURE_ACCOUNT_NAME.blob.core.windows.net/orc/sales-positions/*' - DATA_FORMAT = 'ORC' - AZURE_ACCOUNT_NAME = 'AZURE_ACCOUNT_NAME' - AZURE_SECRET_KEY = 'AZURE_SECRET_KEY' - PARALLELISM = 'nproc()'; -``` - -### Export to Blob Storage - -```sql -EXPORT SALES_POSITIONS -INTO SCRIPT ETL.EXPORT_PATH WITH - BUCKET_PATH = 'wasbs://CONTAINER@AZURE_ACCOUNT_NAME.blob.core.windows.net/parquet/sales-positions/' - AZURE_ACCOUNT_NAME = 'AZURE_ACCOUNT_NAME' - AZURE_SECRET_KEY = 'AZURE_SECRET_KEY' - PARALLELISM = 'iproc()'; -``` - -Furthermore, you can use Shared Access Signature (SAS) token to access the Azure -Blob Storage. In that case, you need to provide the Azure SAS token and -container name as parameters. - -For example: -```sql -IMPORT INTO SALES_POSITIONS -FROM SCRIPT ETL.IMPORT_PATH WITH - BUCKET_PATH = 'wasbs://AZURE_CONTAINER_NAME@AZURE_ACCOUNT_NAME.blob.core.windows.net/orc/sales-positions/*' - DATA_FORMAT = 'ORC' - AZURE_ACCOUNT_NAME = 'AZURE_ACCOUNT_NAME' - AZURE_CONTAINER_NAME = 'AZURE_CONTAINER_NAME' - AZURE_SAS_TOKEN = 'AZURE_SAS_TOKEN' - PARALLELISM = 'nproc()'; -``` - -## Azure Data Lake (Gen1) Storage - -Similarly the following parameters and values are required in order to access -the Azure Data Lake (Gen1) Storage system. - -- `BUCKET_PATH`, that starts with ``adl`` -- `CONTAINER` the blob storage container name -- `AZURE_CLIENT_ID`, this sometimes called also Azure app id -- `AZURE_CLIENT_SECRET` -- `AZURE_DIRECTORY_ID` - -You can follow -[these](https://docs.microsoft.com/en-us/azure/data-lake-store/data-lake-store-service-to-service-authenticate-using-active-directory) -[instructions](https://docs.microsoft.com/en-us/azure/active-directory/develop/howto-create-service-principal-portal) -to obtain the above configuration settings. - -**Please ensure that the above client id has an access permission to the Gen1 -storage container and its sub folders**. - -### Import from Data Lake (Gen1) Store - -```sql -IMPORT INTO SALES_POSITIONS -FROM SCRIPT ETL.IMPORT_PATH WITH - BUCKET_PATH = 'adl://CONTAINER.azuredatalakestore.net/orc/sales_positions/*' - DATA_FORMAT = 'ORC' - AZURE_CLIENT_ID = 'AZURE_CLIENT_ID' - AZURE_CLIENT_SECRET = 'AZURE_CLIENT_SECRET' - AZURE_DIRECTORY_ID = 'AZURE_DIRECTORY_ID' - PARALLELISM = 'nproc()'; -``` - -### Export to Data Lake (Gen1) Store - -```sql -EXPORT SALES_POSITIONS -INTO SCRIPT ETL.EXPORT_PATH WITH - BUCKET_PATH = 'adl://CONTAINER.azuredatalakestore.net/parquet/sales_positions/' - AZURE_CLIENT_ID = 'AZURE_CLIENT_ID' - AZURE_CLIENT_SECRET = 'AZURE_CLIENT_SECRET' - AZURE_DIRECTORY_ID = 'AZURE_DIRECTORY_ID' - PARALLELISM = 'iproc()'; -``` diff --git a/docs/storage/cloud_storages.md b/docs/storage/cloud_storages.md new file mode 100644 index 00000000..de32fd5f --- /dev/null +++ b/docs/storage/cloud_storages.md @@ -0,0 +1,295 @@ +# Cloud Storage Systems + +This guide provides the instructions to setup or configure supported cloud +storage systems, usage examples to transfer data stored in supported file +formats. + +## Table of contents + +- [Prerequisites](#prerequisites) +- [Amazon S3](#amazon-s3) +- [Google Cloud Storage](#google-cloud-storage) +- [Azure Blob Storage](#azure-blob-storage) +- [Azure Data Lake (Gen1) Storage](#azure-blob-storage) + +## Prerequisites + +Before start, you should deploy the latest jar file to the Exasol BucketFS +bucket and create UDF scripts for importing or exporing data from cloud storage +filesystems. Please follow the instructions from [deployment +guide](docs/deployment_guide.md) if you have not done so. + +Additionally, you can read the [user guide](docs/user_guide.md) in order to get +familiar with cloud-storage-etl-udfs in general. + +In this guide, we will be using `RETAIL` schema and `SALES_POSITIONS` table in +examples. + +## Amazon S3 + +Amazon Web Services uses security credentials, for example Access Key and Secret +Key, in order to verify and check whether you have permission to acces a +resource such S3. + +Thus, if you want to import data files from an Amazon Simple Storage Service +(Amazon S3) bucket, your credentials must allow that access. + +Thereofore, in order to access the Amazon S3 bucket you should provide the +following property keys to the UDF scripts with values from AWS credentials +access key and secret key. + +- `S3_ACCESS_KEY` +- `S3_SECRET_KEY` + +Please follow the [Amazon credentials management best practices][aws-creds] when +creating credentials. + +Additionally, you should provide S3 endpoint: + +- `S3_ENDPOINT`, for example, s3.eu-central-1.amazonaws.com. + +An endpoint is the URL of the entry point for an AWS resource. For example, +`https://dynamodb.us-west-2.amazonaws.com` is the endpoint for the Amazon +DynamoDB service in the US West (Oregon) Region. + +### Import from S3 + +```sql +IMPORT INTO RETAIL.SALES_POSITIONS +FROM SCRIPT ETL.IMPORT_PATH WITH + BUCKET_PATH = 's3a:///data/orc/sales_positions/*' + DATA_FORMAT = 'ORC' + S3_ACCESS_KEY = '' + S3_SECRET_KEY = '' + S3_ENDPOINT = 's3..amazonaws.com' + PARALLELISM = 'nproc()*2'; +``` + +### Export to S3 + +```sql +EXPORT RETAIL.SALES_POSITIONS +INTO SCRIPT ETL.EXPORT_PATH WITH + BUCKET_PATH = 's3a:///data/parquet/sales_positions/' + DATA_FORMAT = 'PARQUET' + S3_ACCESS_KEY = '' + S3_SECRET_KEY = '' + S3_ENDPOINT = 's3..amazonaws.com' + PARALLELISM = 'iproc(), floor(random()*2)'; +``` + +Currently, we only support `s3a` URI scheme; therefore, `BUCKET_PATH` should +start with it. + +## Google Cloud Storage + +Similar to Amazon S3, you need to provide security credentials to the UDF in +order to access Google Cloud Storage. + +For this, you need to set two property keys when running UDF: + +- ``GCS_PROJECT_ID`` +- ``GCS_KEYFILE_PATH`` + +The **GCS_PROJECT_ID** is a Google Cloud Platform (GCP) project identifier. It +is unique string for you project which is composed of the project name and a +randomly assigned number. Please check out the GCP [creating and managing +projects][gcp-projects] page for more information. + +The **GCS_KEYFILE_PATH** is path to the location of GCP service account private +key file, usually stored in JSON format. + +A Google Cloud Platform service account is an identity that an instance or an +application can use to authenticate its identity and perform authorized tasks on +Google cloud resources. It is special type of Google account intended to +represent a non-human user that needs to authenticate and be authorized to +access Google APIs. Please check out the GCP [introduction to service +accounts][gcp-auth-intro], [understanding service accounts][gcp-auth-under] and +generating [service account private key][gcp-auth-keys] documentation pages. + +Once the service account is generated, give enough permissions to it to access +the Google Cloud Storage objects and download its private key as JSON file. + +To be able use the service account private key in the UDF, it should be uploaded +to the bucket in Exasol BucketFS. + +Upload GCP service account keyfile: + +```bash +curl -X PUT -T gcp--service-keyfile.json \ + http://w:write-password@exasol.datanode.domain.com:2580//gcp--service-keyfile.json +``` + +Please make sure that the bucket is **secure** and only **readable by users** +who run the UDFs. + +### Import from GCS + +```sql +IMPORT INTO RETAIL.SALES_POSITIONS +FROM SCRIPT ETL.IMPORT_PATH WITH + BUCKET_PATH = 'gs:///data/avro/sales_positions/*' + DATA_FORMAT = 'AVRO' + GCS_PROJECT_ID = '' + GCS_KEYFILE_PATH = '/buckets/bfsdefault//gcp--service-keyfile.json' + PARALLELISM = 'nproc()*4'; +``` + +### Export to GCS + +```sql +EXPORT RETAIL.SALES_POSITIONS +INTO SCRIPT ETL.EXPORT_PATH WITH + BUCKET_PATH = 'gs:///data/parquet/sales_positions/' + DATA_FORMAT = 'PARQUET' + GCS_PROJECT_ID = '' + GCS_KEYFILE_PATH = '/buckets/bfsdefault//gcp--service-keyfile.json' + PARALLELISM = 'iproc(), floor(random()*4)'; +``` + +You should notice that UDF access GCS object using `gs` URI scheme. + +The Exasol user defined functions have access to buckets from predefined file +path, `/buckets/bfsdefault/`. As a result, you can refer to your buckets when +running the cloud-storage-etl-udfs by providing a path to it such as +`/buckets/bfsdefault//`. Please check out the Exasol manual to +learn more about Exasol synchronous cluster filesystem BucketFS. + +## Azure Blob Storage + +Azure Blob Storage containers can be accessed using two possible authotization +mechanism. + +- ``AZURE_SECRET_KEY`` +- ``AZURE_SAS_TOKEN`` + +The **AZURE_SECRET_KEY** is 512-bit storage access keys that can be generated +after creating a storage account. It is used to authorize access to the storage +accounts. + +THE **AZURE_SAS_TOKEN** is Shared Access Signature (SAS) that provides secure +access to storage account with granular control over how the clients can access +the data. + +You should provider either one of these parameters when using +cloud-storage-elt-udfs to access the Azure Blob Storage containers. + +Additionally, you need to obtain the Azure Blob store account name and container +name and provide them as UDF parameters. + +- ``AZURE_ACCOUNT_NAME`` +- ``AZURE_CONTAINER_NAME`` + +The **AZURE_CONTAINER_NAME** parameter is optional if you are using storage +account access keys. However, it should still be available in the +``BUCKET_PATH`` property value string. + +Please refer to Azure documentation on [creating storage +account][azure-blob-account], managing [storage access keys][azure-blob-keys] +and using [shared access signatures (SAS)][azure-blob-sas]. + +### Import from Blob Storage using access key + +```sql +IMPORT INTO RETAIL.SALES_POSITIONS +FROM SCRIPT ETL.IMPORT_PATH WITH + BUCKET_PATH = 'wasbs://@.blob.core.windows.net/data/orc/sales-positions/*' + DATA_FORMAT = 'ORC' + AZURE_ACCOUNT_NAME = '' + AZURE_SECRET_KEY = '' + PARALLELISM = 'nproc()'; +``` + +### Import from Blob Storage using SAS token + +```sql +IMPORT INTO RETAIL.SALES_POSITIONS +FROM SCRIPT ETL.IMPORT_PATH WITH + BUCKET_PATH = 'wasbs://@.blob.core.windows.net/data/orc/sales-positions/*' + DATA_FORMAT = 'ORC' + AZURE_ACCOUNT_NAME = '' + AZURE_CONTAINER_NAME = '' + AZURE_SAS_TOKEN = '' + PARALLELISM = 'nproc()'; +``` + +### Export to Blob Storage + +```sql +EXPORT RETAIL.SALES_POSITIONS +INTO SCRIPT ETL.EXPORT_PATH WITH + BUCKET_PATH = 'wasbs://@.blob.core.windows.net/data/parquet/sales-positions/' + DATA_FORMAT = 'PARQUET' + AZURE_ACCOUNT_NAME = '' + AZURE_SECRET_KEY = '' + PARALLELISM = 'iproc()'; +``` + +Similar to import, you can also use the SAS token when exporting. + +The Azure Blob Storage container path URI scheme can be `wasbs` or `wasb`. + +## Azure Data Lake Storage + +Currently only Azure Data Lake Storage Gen1 version is supported. + +The following properties should be provided to the UDF in order to access the +Azure Data Lake (Gen1) Storage. + +- `AZURE_CLIENT_ID` +- `AZURE_CLIENT_SECRET` +- `AZURE_DIRECTORY_ID` + +The **AZURE_CLIENT_ID** is the Azure Active Directory (AD) App registation +Application ID. + +The **AZURE_CLIENT_SECRET** is the secret key generated for the Application ID. + +The **AZURE_DIRECTORY_ID** is ht Active Directory (AD) Directory (Tenant) ID. + +Please check out the Azure documentation on how to create [service to service +authentication using Active Directory][azure-adl-s2s-auth] and [Azure AD +application and service principal][azure-adl-srv-prin]. These Azure +documentation pages should show how obtain required configuration settings. + +Finally, make sure that the client id has an access permissions to the Gen1 +storage container or its child directories. + +### Import from Data Lake (Gen1) Storage + +```sql +IMPORT INTO RETAIL.SALES_POSITIONS +FROM SCRIPT ETL.IMPORT_PATH WITH + BUCKET_PATH = 'adl://.azuredatalakestore.net/data/avro/sales_positions/*' + DATA_FORMAT = 'AVRO' + AZURE_CLIENT_ID = '' + AZURE_CLIENT_SECRET = '' + AZURE_DIRECTORY_ID = '' + PARALLELISM = 'nproc()'; +``` + +### Export to Data Lake (Gen1) Storage + +```sql +EXPORT RETAIL.SALES_POSITIONS +INTO SCRIPT ETL.EXPORT_PATH WITH + BUCKET_PATH = 'adl://.azuredatalakestore.net/data/parquet/sales_positions/' + DATA_FORMAT = 'PARQUET' + AZURE_CLIENT_ID = '' + AZURE_CLIENT_SECRET = '' + AZURE_DIRECTORY_ID = '' + PARALLELISM = 'iproc()'; +``` + +The container path should start with `adl` URI scheme. + +[aws-creds]: https://docs.aws.amazon.com/general/latest/gr/aws-sec-cred-types.html +[gcp-projects]: https://cloud.google.com/resource-manager/docs/creating-managing-projects +[gcp-auth-intro]: https://cloud.google.com/compute/docs/access/service-accounts +[gcp-auth-under]: https://cloud.google.com/iam/docs/understanding-service-accounts +[gcp-auth-keys]: https://cloud.google.com/video-intelligence/docs/common/auth +[azure-blob-account]: https://docs.microsoft.com/en-us/azure/storage/common/storage-quickstart-create-account +[azure-blob-keys]: https://docs.microsoft.com/en-us/azure/storage/common/storage-account-manage#access-keys +[azure-blob-sas]: https://docs.microsoft.com/en-us/azure/storage/common/storage-sas-overview +[azure-adl-s2s-auth]: https://docs.microsoft.com/en-us/azure/data-lake-store/data-lake-store-service-to-service-authenticate-using-active-directory +[azure-adl-src-prin]: https://docs.microsoft.com/en-us/azure/active-directory/develop/howto-create-service-principal-portal diff --git a/docs/user_guide.md b/docs/user_guide.md new file mode 100644 index 00000000..3df70ece --- /dev/null +++ b/docs/user_guide.md @@ -0,0 +1,282 @@ +# User Guide + +This section provides short instructions and examples so that you can get +started with using the cloud-storage-etl-udfs. + +## Table of contents + +- [Deployment Guide](#deployment-guide) +- [Getting Started](#getting-started) +- [Usage](#usage) +- [Properties](#properties) + - [Storage Properties](#storage-properties) + - [Kafka Import Properties](#kafka-import-properties) + +## Deployment guide + +In order to use the cloud-storage-etl-udfs, you should deploy the jar file and +create UDF scripts. + +Please follow the steps from [deployment guide](docs/deployment_guide.md). + +## Getting started + +The cloud-storage-etl-udfs allows you to import data from external cloud storage +filesystems or Apache Kafka cluster into Exasol table. Similarly, it allows you +to export data from Exasol table to cloud storage filesystems. + +Therefore, the next step is to create a table inside the Exasol. + +In this guide, we are going to create a `RETAIL` schema with `SALES_POSITIONS` +table. Then we will use this table to demonstrate the import and export +features of the cloud-storage-etl-udfs. + +Create a retail schema: + +```sql +CREATE SCHEMA RETAIL; +``` + +Then create a sales positions table: + +```sql +OPEN SCHEMA RETAIL; + +CREATE OR REPLACE TABLE SALES_POSITIONS ( + SALES_ID INTEGER, + POSITION_ID SMALLINT, + ARTICLE_ID SMALLINT, + AMOUNT SMALLINT, + PRICE DECIMAL(9,2), + VOUCHER_ID SMALLINT, + CANCELED BOOLEAN +); +``` + +## Usage + +Here we are going to show an excerpt from a simple example that will import Avro +formatted data from Azure Blob Storage into `SALES_POSITIONS` table and export +from it into Amazon S3 bucket as Parquet formatted data. + +Import Avro data from Azure Blob store container: + +```sql +IMPORT INTO RETAIL.SALES_POSITIONS +FROM SCRIPT ETL.IMPORT_PATH WITH + BUCKET_PATH = 'wasbs://@.blob.core.windows.net/data/avro/sales-positions/*' + DATA_FORMAT = 'AVRO' + AZURE_ACCOUNT_NAME = '' + AZURE_SECRET_KEY = '' + PARALLELISM = 'nproc()'; +``` + +When running the import, it will first execute the `IMPORT_METADATA` udf in +order to calculate the number of files in the user provided path. Then it will +distribute these files in round-robin fashion into number of `PARALLELISM` +virtual machines that then will be importing data from their list of files. + +In the example above, `PARALLELISM` property value is set to `nproc()` which +defines the number of physical datanodes in the Exasol cluster. Therefore, each +datanode will be running single virtual machine (VM) to import data from files. + +However, you can increase the parallelism by setting it to different value. For +example, in order to start four VMs in each datanode, you can set the +`PARALLELISM` to `nproc()*4`. Or similarly, to higher static number as +`PARALLELISM = '16'` that will use 16 virtual machines in total. + +Once the process is finished, make sure that data is imported: + +```sql +SELECT * FROM retail.sales_positions LIMIT 10; + +SELECT count(*) FROM retail.sales_positions; +``` + +Similarly, to export table rows into Amazon S3 bucket as Parquet data: + +```sql +EXPORT RETAIL.SALES_POSITIONS +INTO SCRIPT ETL.EXPORT_PATH WITH + BUCKET_PATH = 's3a:///data/parquet/sales_positions/' + DATA_FORMAT = 'PARQUET' + S3_ACCESS_KEY = '' + S3_SECRET_KEY = '' + S3_ENDPOINT = 's3..amazonaws.com' + PARALLELISM = 'iproc(), floor(random()*4)'; +``` + +Under the hoods, the `EXPORT_PATH` entry point UDF will call the `EXPORT_TABLE` +user defined function we created in the deployment guide. + +You might have noticed the different value for `PARALLELISM` in the export SQL +statement. This is due to fact that when exporting an Exasol table, the +`PARALLELISM` property value will be used internally in a `GROUP BY` clause to +distribute table rows into many virtual machines. + +In the example above, `iproc()` will return the sequence numbers of Exasol +datanodes. Thus, setting parallelism to `iproc()` only will create a single VM +per each physical datanode. Therefore, we can add additional number to `iproc()` +to increase parallel running VMs for exporting data. + +Please be aware that setting the `PARALLELISM` to a static number, for instance +`16` or `nproc()`, when exporting will only start a single virtual machine in +total. + +Finally, please change the placeholder parameters accordingly for your use case. + +## Properties + +Several properties should be provided by users in order `cloud-storage-etl-udfs` +to work properly. As you have seen from examples they are required parameters +such as path to storage filesystem, access control credentials or optional +parameters such as parallelism. + +Please note that properties are provided as strings, `S3_ENDPOINT = +'s3.eu-central-1.amazonaws.com'`. + +### Storage Properties + +These property names related to accessing various cloud storage filesystems for +importing or exporting data. + +#### Required Properties + +* ``BUCKET_PATH`` - It specifies a path to the cloud storage filesystem. + Additionally, it should start with storage specific schema, such as `s3a`. You + can check out the currently [supported list of cloud storage + systems](docs/storage/cloud_storages.md). + +* ``DATA_FORMAT`` - It defines the data file format in the user provided path. + Currently, we support importing data from **Avro**, **Orc** and **Parquet** + file formats and exporting to only **Parquet** format. + +* Additional storage related properties that enable accessing storage + filesystems. Please refer to the [supported cloud storage + systems](docs/storage/cloud_storages.md) documentation for more information. + +#### Optional Properties + +These are optional parameters that usually have default values. + +* ``PARALLELISM`` - It defines the number of parallel virtual-machine instances + that will be started to import or export data. Default value in import SQL + statement: `nproc()`. Default value in export SQL statement: `iproc()`. As + mentioned before, you can changes these values to increase the parallelism. + For example, multiply with a number when importing `PARALLELISM = 'nproc()*4'` + or append additional numbers when exporting `PARALLELISM = 'iproc(), + floor(random()*4)'`. Please check out the [supported cloud storage + systems](docs/storage/cloud_storages.md) for more examples. + +* ``PARQUET_COMPRESSION_CODEC`` - This property is only used in export SQL + statement. It defines the compression codec to use when exporting data into + Parquet formatted files. Default value is **uncompressed**. Other compression + options are **snappy**, **gzip** and **lzo**. + +* ``EXPORT_BATCH_SIZE`` - This property is only used in export SQL statement. It + defines the number of records per file from each virtual machine. Default + value is **100000**. That is, if a single VM gets `1M` rows to export, it will + create 10 files with default value 100000 records in each file. + +### Kafka Import Properties + +The following properties are related to UDFs when importing data from Kafka +clusters. Most of these properties are exactly same as [Kafka consumer +configurations][kafka-consumer-configs]. + +#### Required Properties + +* ``BOOTSTRAP_SERVERS`` - It is a comma separated host-port pairs of Kafka + brokers. These addresses will be used to establish the initial connection to + the Kafka cluster. + +* ``SCHEMA_REGISTRY_URL`` - It specifies an URL to the Confluent [Schema + Registry][schema-registry] which stores Avro schemas as metadata. Schema + Registry will be used to parse the Kafka topic Avro data schemas. + +* ``TOPICS`` - It defines Kafka topic name that we want to import data from. + Currently, we only support single topic data imports. Therefore, it should not + contain comma separated list of more than one topic names. + +* ``TABLE_NAME`` - It defines the Exasol table name the data will be imported. + This is required as user provided parameter since unfortunately we cannot + obtain table name from inside UDF even though we are importing data into it. + +Please note that we do not have `PARALLELISM` when importing from Kafka cluster. +The number of parallel running virtual machine instances are defined by the +Kafka topic partitions. That is, when importing data from Kafka topic, we will +be importing from each topic partition in parallel. Therefore, it is important +to configure Kafka topics with several partitions. + +Please check out the [Kafka import examples](docs/kafka/import.md) for more +information. + +#### Optional Properties + +These are optional parameters with their default values. + +* ``GROUP_ID`` - It defines the id for this type of consumers. Default value is + **EXASOL_KAFKA_UDFS_CONSUMERS**. It is a unique string that identifies the + consumer group this consumer belongs to. + +* ``POLL_TIMEOUT_MS`` - It defines the timeout value that is the number of + milliseconds to wait for the consumer poll function to return any data. + Default value is **30000** milliseconds. + +* ``MIN_RECORDS_PER_RUN`` - It is an upper bound on the minimum number of + records to consumer per UDF run. Default value is **100**. That is, if the + pull function returns fewer records than this number, we consume returned + records and finish the UDF process. Otherwise, we continue polling more data + until total number of records reaches certain threshold, for example, + `MAX_RECORD_PER_RUN`. + +* ``MAX_RECORD_PER_RUN`` - It is a lower bound on the maximum number of records + to consumer per UDF run. Default value is **1000000**. When the returned + number of records from poll is more than `MIN_RECORDS_PER_RUN`, we continue + polling for more records until total number reaches this number. + +* ``MAX_POLL_RECORDS`` - It is the maximum number of records returned in a + single call from the consumer poll method. Default value is **500**. + +* ``FETCH_MIN_BYTES`` - It is the minimum amount of data the server should + return for a fetch request. If insufficient data is available the request will + wait for that much data to accumulate before answering the request. Default + value is **1**. + +* ``FETCH_MAX_BYTES`` - It is the maximum amount of data the server should + return for a fetch request. Default value is **52428800**. + +* ``MAX_PARTITION_FETCH_BYTES`` - It is the maximum amount of data per + partition the server will return. Default value is **1048576**. + +The following properties should be provided to enable secure connection to the +Kafka clusters. + +* ``SSL_ENABLED`` - It is a boolean property that should be set to `true` in + order to use the secure connections to the Kafka cluster. Default value is + **'false'**. + +* ``SECURITY_PROTOCOL`` - It is the protocol used to communicate with Kafka + servers. Default value is **PLAINTEXT**. + +* ``SSL_KEY_PASSWORD`` - It represents the password of the private key inside + the keystore file. + +* ``SSL_KEYSTORE_PASSWORD`` - It the store password for the keystore file. + +* ``SSL_KEYSTORE_LOCATION`` - It represents the location of the keystore file. + This location value should point to the keystore file that is available via + Exasol bucket in BucketFS. + +* ``SSL_TRUSTSTORE_PASSWORD`` - It is the password for the truststore file. + +* ``SSL_TRUSTSTORE_LOCATION`` - It is the location of the truststore file, and + it should refer to the truststore file stored inside bucket in Exasol + BucketFS. + +* ``SSL_ENDPOINT_IDENTIFICATION_ALGORITHM`` - It is the endpoint identification + algorithm to validate server hostname using server certificate. Default value + is **https**. + +[schema-registry]: https://docs.confluent.io/current/schema-registry/index.html +[kafka-consumer-configs]: https://kafka.apache.org/documentation/#consumerconfigs diff --git a/project/Settings.scala b/project/Settings.scala index 83d5d5bb..a18c86e0 100644 --- a/project/Settings.scala +++ b/project/Settings.scala @@ -46,8 +46,8 @@ object Settings { coverageMinimum := 50, coverageOutputHTML := true, coverageOutputXML := true, + coverageOutputCobertura := true, coverageFailOnMinimum := false, - coverageOutputCobertura := false, // Git versioning, use git describe git.useGitDescribe := true ) diff --git a/project/plugins.sbt b/project/plugins.sbt index 28e6e856..f3e4deba 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -14,7 +14,8 @@ addSbtPlugin("org.wartremover" % "sbt-wartremover-contrib" % "1.2.4") // http://github.com/danielnixon/extrawarts addSbtPlugin("org.danielnixon" % "sbt-extrawarts" % "1.0.3") -// Adds a `assembly` task to create a fat JAR with all of its dependencies +// Adds a `assembly` task to create a fat JAR with all of its +// dependencies // https://github.com/sbt/sbt-assembly addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.9") @@ -30,7 +31,13 @@ addSbtPlugin("com.thoughtworks.sbt-api-mappings" % "sbt-api-mappings" % "2.1.0") // http://github.com/scoverage/sbt-scoverage addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.5.1") -// Adds a `dependencyUpdates` task to check Maven repositories for dependency updates +// Adds SBT Coveralls plugin for uploading Scala code coverage to +// https://coveralls.io +// https://github.com/scoverage/sbt-coveralls +addSbtPlugin("org.scoverage" % "sbt-coveralls" % "1.2.7") + +// Adds a `dependencyUpdates` task to check Maven repositories for +// dependency updates // http://github.com/rtimush/sbt-updates addSbtPlugin("com.timushev.sbt" % "sbt-updates" % "0.4.0") @@ -62,7 +69,8 @@ addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "1.0.0") // https://github.com/cb372/sbt-explicit-dependencies addSbtPlugin("com.github.cb372" % "sbt-explicit-dependencies" % "0.2.9") -// Setup this and project/project/plugins.sbt for formatting project/*.scala files with scalafmt +// Setup this and project/project/plugins.sbt for formatting +// project/*.scala files with scalafmt inThisBuild( Seq( scalafmtOnCompile := true,