From 041d34e94ca982c55bdb664d94d8216fa4f21e3e Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 16 Jun 2022 15:45:28 +0200 Subject: [PATCH 01/34] Implement KafkaService --- .../api/rest/services/KafkaService.scala | 98 +++++++++++++++ .../api/rest/services/KafkaServiceTest.scala | 117 ++++++++++++++++++ 2 files changed, 215 insertions(+) create mode 100644 src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala create mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala new file mode 100644 index 000000000..349165f53 --- /dev/null +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -0,0 +1,98 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} +import org.apache.kafka.common.TopicPartition +import org.slf4j.LoggerFactory +import org.springframework.stereotype.Service +import za.co.absa.hyperdrive.trigger.configuration.application.{GeneralConfig, KafkaConfig} +import za.co.absa.hyperdrive.trigger.models.enums.JobTypes +import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanceParameters} + +import java.util.Properties +import javax.inject.Inject +import scala.collection.mutable + +trait KafkaService { + def getKafkaProperties(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] + def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] +} + +@Service +class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) extends KafkaService { + private val logger = LoggerFactory.getLogger(this.getClass) + private val kafkaConsumersCache: mutable.Map[Properties, KafkaConsumer[String, String]] = mutable.Map() + private val HyperdriveKafkaTopicKey = "reader.kafka.topic" + private val HyperdriveKafkaBrokersKey = "reader.kafka.brokers" + private val HyperdriveKafkaExtraOptionsKey = "reader.option.kafka" + + override def getKafkaProperties(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] = { + if (jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || + !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters]) { + None + } else { + val args = jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters].appArguments + val topicOpt = args.find(_.startsWith(s"$HyperdriveKafkaTopicKey=")) + .map(_.replace(s"$HyperdriveKafkaTopicKey=", "")) + val kafkaBrokersOpt = args.find(_.startsWith(s"$HyperdriveKafkaBrokersKey=")) + .map(_.replace(s"$HyperdriveKafkaBrokersKey=", "")) + val kafkaArgs = args.filter(_.startsWith(s"$HyperdriveKafkaExtraOptionsKey.")) + .map(_.replace(s"$HyperdriveKafkaExtraOptionsKey.", "")) + .filter(_.contains("=")) + .map { s => + val keyValue = s.split("=", 2) + val key = keyValue(0).trim + val value = keyValue(1).trim + (key, value) + } + .toMap + val properties = new Properties() + kafkaArgs.foreach { case (key, value) => properties.setProperty(key, value)} + + (topicOpt, kafkaBrokersOpt) match { + case (Some(topic), Some(kafkaBrokers)) => + properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaBrokers) + Some(topic, properties) + case (_, _) => + logger.warn(s"Topic and/or bootstrap servers could not be found in app arguments for jobDefinition" + + s" ${jobDefinition.name}. Topic: ${topicOpt}, Bootstrap servers: ${kafkaBrokersOpt}") + None + } + } + } + + override def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] = { + val groupId = s"${kafkaConfig.groupIdPrefix}-${generalConfig.appUniqueId}-getEndOffsets" + consumerProperties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + val consumer = kafkaConsumersCache.getOrElse(consumerProperties, { + val consumer = new KafkaConsumer[String, String](consumerProperties) + kafkaConsumersCache.put(consumerProperties, consumer) + consumer + }) + + import scala.collection.JavaConverters._ + val partitionInfo = consumer.partitionsFor(topic).asScala + val topicPartitions = partitionInfo.map(p => new TopicPartition(p.topic(), p.partition())) + consumer.endOffsets(topicPartitions.asJava) + .asScala + .map { + case (topicPartition: TopicPartition, offset: java.lang.Long) => topicPartition.partition() -> offset.longValue() + } + .toMap + } +} diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala new file mode 100644 index 000000000..0ecd89be4 --- /dev/null +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala @@ -0,0 +1,117 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.scalatest.{FlatSpec, Matchers} +import za.co.absa.hyperdrive.trigger.configuration.application.{TestGeneralConfig, TestKafkaConfig} +import za.co.absa.hyperdrive.trigger.models.enums.JobTypes +import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, ShellInstanceParameters, SparkInstanceParameters} + +class KafkaServiceTest extends FlatSpec with Matchers { + val underTest = new KafkaServiceImpl(TestKafkaConfig(), TestGeneralConfig()) + + "getKafkaProperties" should "get kafka properties from a resolved job definition" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "reader.option.kafka.security.protocol=PLAINTEXT" + ) + ), + order = 1 + ) + + val result = underTest.getKafkaProperties(jobDefinition) + + result.isDefined shouldBe true + result.get._1 shouldBe "some-topic" + val properties = result.get._2 + properties.getProperty("bootstrap.servers") shouldBe "http://localhost:9092" + properties.getProperty("security.protocol") shouldBe "PLAINTEXT" + } + + it should "return None if the topic is not in the app arguments" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.brokers=http://localhost:9092") + ), + order = 1 + ) + + val result = underTest.getKafkaProperties(jobDefinition) + + result shouldBe None + } + + it should "return None if the kafka brokers are not in the app arguments" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.topic=some-topic") + ), + order = 1 + ) + + val result = underTest.getKafkaProperties(jobDefinition) + + result shouldBe None + } + + it should "return None if the job type is not Hyperdrive" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Spark, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.topic=some-topic", "reader.kafka.brokers=http://localhost:9092") + ), + order = 1 + ) + + val result = underTest.getKafkaProperties(jobDefinition) + + result shouldBe None + } + + it should "return None if the jobParameters are not SparkInstanceParameters" in { + val jobDefinition = ResolvedJobDefinition( + "inconsistentJob", + ShellInstanceParameters( + jobType = JobTypes.Hyperdrive, + scriptLocation = "script.sh" + ), + order = 1 + ) + + val result = underTest.getKafkaProperties(jobDefinition) + + result shouldBe None + } +} From 8e175c96954116b042c157dabf45f2c7ce2a6d96 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 16 Jun 2022 15:45:38 +0200 Subject: [PATCH 02/34] Implement KafkaService --- .../absa/hyperdrive/trigger/api/rest/services/KafkaService.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index 349165f53..625684e25 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -1,4 +1,3 @@ - /* * Copyright 2018 ABSA Group Limited * From 8166509d674c407b31382bd2d23a5ef9e26b9f76 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 16 Jun 2022 15:47:27 +0200 Subject: [PATCH 03/34] formatting --- .../api/rest/services/KafkaService.scala | 46 +++++++++++-------- 1 file changed, 28 insertions(+), 18 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index 625684e25..fa2fa9a3c 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -28,8 +28,8 @@ import javax.inject.Inject import scala.collection.mutable trait KafkaService { - def getKafkaProperties(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] - def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] + def getKafkaProperties(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] + def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] } @Service @@ -41,16 +41,21 @@ class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: Gener private val HyperdriveKafkaExtraOptionsKey = "reader.option.kafka" override def getKafkaProperties(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] = { - if (jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || - !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters]) { + if ( + jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || + !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] + ) { None } else { val args = jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters].appArguments - val topicOpt = args.find(_.startsWith(s"$HyperdriveKafkaTopicKey=")) + val topicOpt = args + .find(_.startsWith(s"$HyperdriveKafkaTopicKey=")) .map(_.replace(s"$HyperdriveKafkaTopicKey=", "")) - val kafkaBrokersOpt = args.find(_.startsWith(s"$HyperdriveKafkaBrokersKey=")) + val kafkaBrokersOpt = args + .find(_.startsWith(s"$HyperdriveKafkaBrokersKey=")) .map(_.replace(s"$HyperdriveKafkaBrokersKey=", "")) - val kafkaArgs = args.filter(_.startsWith(s"$HyperdriveKafkaExtraOptionsKey.")) + val kafkaArgs = args + .filter(_.startsWith(s"$HyperdriveKafkaExtraOptionsKey.")) .map(_.replace(s"$HyperdriveKafkaExtraOptionsKey.", "")) .filter(_.contains("=")) .map { s => @@ -61,15 +66,17 @@ class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: Gener } .toMap val properties = new Properties() - kafkaArgs.foreach { case (key, value) => properties.setProperty(key, value)} + kafkaArgs.foreach { case (key, value) => properties.setProperty(key, value) } (topicOpt, kafkaBrokersOpt) match { case (Some(topic), Some(kafkaBrokers)) => properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaBrokers) Some(topic, properties) case (_, _) => - logger.warn(s"Topic and/or bootstrap servers could not be found in app arguments for jobDefinition" + - s" ${jobDefinition.name}. Topic: ${topicOpt}, Bootstrap servers: ${kafkaBrokersOpt}") + logger.warn( + s"Topic and/or bootstrap servers could not be found in app arguments for jobDefinition" + + s" ${jobDefinition.name}. Topic: ${topicOpt}, Bootstrap servers: ${kafkaBrokersOpt}" + ) None } } @@ -78,19 +85,22 @@ class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: Gener override def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] = { val groupId = s"${kafkaConfig.groupIdPrefix}-${generalConfig.appUniqueId}-getEndOffsets" consumerProperties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) - val consumer = kafkaConsumersCache.getOrElse(consumerProperties, { - val consumer = new KafkaConsumer[String, String](consumerProperties) - kafkaConsumersCache.put(consumerProperties, consumer) - consumer - }) + val consumer = kafkaConsumersCache + .getOrElse(consumerProperties, { + val consumer = new KafkaConsumer[String, String](consumerProperties) + kafkaConsumersCache.put(consumerProperties, consumer) + consumer + } + ) import scala.collection.JavaConverters._ val partitionInfo = consumer.partitionsFor(topic).asScala val topicPartitions = partitionInfo.map(p => new TopicPartition(p.topic(), p.partition())) - consumer.endOffsets(topicPartitions.asJava) + consumer + .endOffsets(topicPartitions.asJava) .asScala - .map { - case (topicPartition: TopicPartition, offset: java.lang.Long) => topicPartition.partition() -> offset.longValue() + .map { case (topicPartition: TopicPartition, offset: java.lang.Long) => + topicPartition.partition() -> offset.longValue() } .toMap } From 3e2d88cea8f7c1ef90bada8a4cf81d0bdadc9309 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 20 Jun 2022 18:44:19 +0200 Subject: [PATCH 04/34] wip --- pom.xml | 14 ++ .../api/rest/services/HdfsService.scala | 162 ++++++++++++++++++ .../ResolvedJobDefinitionService.scala | 71 ++++++++ .../UserGroupInformationWrapper.scala | 30 ++++ .../api/rest/services/HdfsServiceTest.scala | 125 ++++++++++++++ .../ResolvedJobDefinitionServiceTest.scala | 47 +++++ 6 files changed, 449 insertions(+) create mode 100644 src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala create mode 100644 src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala create mode 100644 src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationWrapper.scala create mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala create mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala diff --git a/pom.xml b/pom.xml index 88588cf26..356a58dc6 100644 --- a/pom.xml +++ b/pom.xml @@ -75,6 +75,7 @@ 42.3.3 2.2.0 2.2.0 + 4.6.0 2.7.3 2.0.3 2.0.2 @@ -108,6 +109,7 @@ 1.7 6.2.0.Final 1.12.29 + 1.0.3 1.6 @@ -184,6 +186,12 @@ kafka_${scala.compat.version} ${kafka.version} + + + za.co.absa.hyperdrive + driver_${scala.compat.version} + ${hyperdrive.version} + @@ -394,6 +402,12 @@ ${embedded.kafka.version} test + + za.co.absa.commons + commons_${scala.compat.version} + ${absa.commons.version} + test + diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala new file mode 100644 index 000000000..9af7555d6 --- /dev/null +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -0,0 +1,162 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import org.apache.commons.io.IOUtils +import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} +import org.apache.spark.deploy.SparkHadoopUtil +import org.slf4j.LoggerFactory +import org.springframework.stereotype.Service + +import java.nio.charset.StandardCharsets.UTF_8 +import javax.inject.Inject +import scala.io.Source + +trait HdfsService { + type TopicPartitionOffsets = Map[String, Map[Int, Long]] + + def getLatestOffsetFile(params: GetLatestOffsetFileParams): Option[(String, Boolean)] + + def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] + + def getLatestCommitBatchId(checkpointDir: String): Option[Long] + + def getLatestOffsetBatchId(checkpointDir: String): Option[Long] +} + +class GetLatestOffsetFileParams ( + val keytab: String, + val principal: String, + val checkpointLocation: String +) + +@Service +class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformationWrapper) extends HdfsService { + private val logger = LoggerFactory.getLogger(this.getClass) + private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) + private lazy val conf = SparkHadoopUtil.get.conf + private lazy val fs = FileSystem.get(conf) + + /** + * See org.apache.spark.sql.execution.streaming.HDFSMetadataLog + */ + private val batchFilesFilter = new PathFilter { + override def accept(path: Path): Boolean = { + try { + path.getName.toLong + true + } catch { + case _: NumberFormatException => + false + } + } + } + + override def getLatestOffsetFile(params: GetLatestOffsetFileParams): Option[(String, Boolean)] = { + userGroupInformationWrapper.loginUserFromKeytab(params.principal, params.keytab) + val commitBatchIdOpt = getLatestCommitBatchId(params.checkpointLocation) + val offsetBatchIdOpt = getLatestOffsetBatchId(params.checkpointLocation) + + offsetBatchIdOpt.map { offsetBatchId => + val committed = commitBatchIdOpt match { + case Some(commitBatchId) => offsetBatchId == commitBatchId + case None => false + } + val path = new Path(s"${params.checkpointLocation}/${offsetBatchId}") + (path.toString, committed) + } + } + + /** + * + * @param pathStr path to the file as a string + * @param parseFn function that parses the file line by line. Caution: It must materialize the content, + * because the file is closed after the method completes. E.g. it must not return an iterator. + * @tparam R type of the parsed value + * @return None if the file doesn't exist, Some with the parsed content + */ + override def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] = { + val path = new Path(pathStr) + if (fs.exists(path)) { + val input = fs.open(path) + try { + val lines = Source.fromInputStream(input, UTF_8.name()).getLines() + Some(parseFn(lines)) + } catch { + case e: Exception => + // re-throw the exception with the log file path added + throw new Exception( + s"Failed to parse file $path. ${e.getMessage}", e) + } finally { + IOUtils.closeQuietly(input) + } + } else { + logger.debug(s"Could not find file $path") + None + } + } + + /** + * see org.apache.spark.sql.execution.streaming.OffsetSeqLog + * and org.apache.spark.sql.kafka010.JsonUtils + * for details on the assumed format + */ + def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets = { + val SERIALIZED_VOID_OFFSET = "-" + def parseOffset(value: String): Option[TopicPartitionOffsets] = value match { + case SERIALIZED_VOID_OFFSET => None + case json => Some(mapper.readValue(json, classOf[TopicPartitionOffsets])) + } + if (!lines.hasNext) { + throw new IllegalStateException("Incomplete log file") + } + + lines.next() // skip version + lines.next() // skip metadata + lines + .map(parseOffset) + .filter(_.isDefined) + .map(_.get) + .toSeq + .head + } + + def getLatestCommitBatchId(checkpointDir: String): Option[Long] = { + val commitsDir = new Path(s"$checkpointDir/commits") + getLatestBatchId(commitsDir) + } + + def getLatestOffsetBatchId(checkpointDir: String): Option[Long] = { + val commitsDir = new Path(s"$checkpointDir/offsets") + getLatestBatchId(commitsDir) + } + + private def getLatestBatchId(path: Path): Option[Long] = { + if (fs.exists(path)) { + fs.listStatus(path, batchFilesFilter).map { + status => status.getPath.getName.toLong + } + .sorted + .lastOption + } else { + logger.debug(s"Could not find path $path") + None + } + } +} diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala new file mode 100644 index 000000000..b143b3493 --- /dev/null +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala @@ -0,0 +1,71 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.springframework.stereotype.Service +import za.co.absa.hyperdrive.driver.drivers.CommandLineIngestionDriver +import za.co.absa.hyperdrive.trigger.configuration.application.SparkConfig +import za.co.absa.hyperdrive.trigger.models.enums.JobTypes +import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanceParameters} + +import javax.inject.Inject + +trait ResolvedJobDefinitionService { + def resolveAppArguments(jobDefinition: ResolvedJobDefinition): ResolvedJobDefinition + def getLatestOffsetFileParams(jobDefinition: ResolvedJobDefinition): Option[GetLatestOffsetFileParams] +} + +@Service +class ResolvedJobDefinitionServiceImpl @Inject()(sparkConfig: SparkConfig) extends ResolvedJobDefinitionService { + private val HyperdriveCheckpointKey = "writer.common.checkpoint.location" + + override def resolveAppArguments(jobDefinition: ResolvedJobDefinition): ResolvedJobDefinition = { + if ( + jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || + !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] + ) { + jobDefinition + } else { + val jobParameters = jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters] + val args = jobParameters.appArguments + val config = CommandLineIngestionDriver.parseConfiguration(args.toArray) + import scala.collection.JavaConverters._ + val resolvedArgs = config.getKeys.asScala.map { + k => s"${k}=${config.getString(k)}" + }.toList + + jobDefinition.copy(jobParameters = jobParameters.copy(appArguments = resolvedArgs)) + } + } + + override def getLatestOffsetFileParams(jobDefinition: ResolvedJobDefinition): Option[GetLatestOffsetFileParams] = { + if ( + jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || + !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] + ) { + None + } else { + for { + keytab <- sparkConfig.yarn.additionalConfs.get("spark.yarn.keytab") + principal <- sparkConfig.yarn.additionalConfs.get("spark.yarn.principal") + checkpointLocation <- jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters].appArguments + .find(_.startsWith(HyperdriveCheckpointKey)) + .map(_.replace(s"$HyperdriveCheckpointKey=", "")) + } yield new GetLatestOffsetFileParams(keytab, principal, checkpointLocation) + } + } +} diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationWrapper.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationWrapper.scala new file mode 100644 index 000000000..60e4e6ba2 --- /dev/null +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationWrapper.scala @@ -0,0 +1,30 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.apache.hadoop.security.UserGroupInformation +import org.springframework.stereotype.Service + +trait UserGroupInformationWrapper { + def loginUserFromKeytab(principal: String, keytab: String): Unit +} + +@Service +class UserGroupInformationWrapperImpl extends UserGroupInformationWrapper { + override def loginUserFromKeytab(principal: String, keytab: String): Unit = + UserGroupInformation.loginUserFromKeytab(principal, keytab) +} diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala new file mode 100644 index 000000000..6f4d23b34 --- /dev/null +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala @@ -0,0 +1,125 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.doNothing +import org.scalatest.mockito.MockitoSugar +import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} +import za.co.absa.commons.io.TempDirectory + +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Path} + + +class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { + private val ugiWrapper = mock[UserGroupInformationWrapper] + doNothing().when(ugiWrapper).loginUserFromKeytab(any(), any()) + private val underTest = new HdfsServiceImpl(ugiWrapper) + private var baseDir: TempDirectory = _ + private var baseDirPath: Path = _ + + before { + baseDir = TempDirectory("HdfsServiceTest").deleteOnExit() + baseDirPath = baseDir.path.toAbsolutePath + } + + after { + baseDir.delete() + } + + "parseFileAndClose" should "successfully parse a file" in { + val tmpFile = Files.createTempFile(baseDirPath, "hdfsServiceTest", "") + val text = Seq("1", "2", "3").mkString("\n") + Files.write(tmpFile, text.getBytes(StandardCharsets.UTF_8)) + val parseFn: Iterator[String] => Seq[Int] = it => it.toList.map(_.toInt) + + val result = underTest.parseFileAndClose(tmpFile.toAbsolutePath.toString, parseFn) + + result.isDefined shouldBe true + result.get should contain theSameElementsAs Seq(1, 2, 3) + } + + it should "return None if the file does not exist" in { + val parseFn: Iterator[String] => Seq[Int] = _ => Seq() + + val result = underTest.parseFileAndClose("non-existent", parseFn) + + result shouldBe None + } + + "parseKafkaOffsetStream" should "parse an offset file" in { + val lines = Seq( + "v1", + raw"""{"batchWatermarkMs":0,"batchTimestampMs":1633360640176}""", + raw"""{"my.topic":{"2":2021,"1":1021,"3":3021,"0":21}, "my.other.topic":{"0":0}}""" + ).toIterator + + val result = underTest.parseKafkaOffsetStream(lines) + + result.size shouldBe 2 + result.head._1 shouldBe "my.topic" + result.head._2 should contain theSameElementsAs Map("2" -> 2021, "1" -> 1021, "3" -> 3021, "0" -> 21) + result.toSeq(1)._1 shouldBe "my.other.topic" + result.toSeq(1)._2 should contain theSameElementsAs Map("0" -> 0) + } + + it should "throw an exception if the file is incomplete" in { + val lines = Seq().toIterator + + val result = the[Exception] thrownBy underTest.parseKafkaOffsetStream(lines) + + result.getMessage shouldBe "Incomplete log file" + } + + "getLatestCommitBatchId" should "get the latest batch id" in { + val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") + val tmpCommitsDir = Files.createDirectory(tmpCheckpointDir.resolve( "commits")) + (0 to 12).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) + + val result = underTest.getLatestCommitBatchId(tmpCheckpointDir.toAbsolutePath.toString) + + result.isDefined shouldBe true + result.get shouldBe 12 + } + + it should "return None if the checkpoints folder does not exist" in { + val result = underTest.getLatestCommitBatchId("non-existent") + + result.isDefined shouldBe false + } + + it should "return None if the commits folder is empty" in { + val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") + val tmpCommitsDir = Files.createDirectory(tmpCheckpointDir.resolve( "commits")) + + val result = underTest.getLatestCommitBatchId(tmpCheckpointDir.toAbsolutePath.toString) + + result.isDefined shouldBe false + } + + "getLatestOffsetBatchId" should "get the latest batch id" in { + val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") + val tmpCommitsDir = Files.createDirectory(tmpCheckpointDir.resolve( "offsets")) + (0 to 7).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) + + val result = underTest.getLatestOffsetBatchId(tmpCheckpointDir.toAbsolutePath.toString) + + result.isDefined shouldBe true + result.get shouldBe 7 + } +} diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala new file mode 100644 index 000000000..1613dbfd9 --- /dev/null +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala @@ -0,0 +1,47 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} +import za.co.absa.hyperdrive.trigger.models.enums.JobTypes +import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanceParameters} + + +class ResolvedJobDefinitionServiceTest extends FlatSpec with Matchers with BeforeAndAfter { + private val underTest = new ResolvedJobDefinitionServiceImpl() + + "resolveAppArguments" should "successfully parse a file" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=my-topic", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + val result = underTest.resolveAppArguments(jobDefinition) + + result.jobParameters.asInstanceOf[SparkInstanceParameters].appArguments should contain theSameElementsAs + Seq("reader.kafka.topic=my-topic", "writer.common.checkpoint.location=/checkpoint/path/my-topic") + } + +} From 55f02f2d47f7395fb445fc32c021fa70c4d672b9 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 22 Jun 2022 11:08:27 +0200 Subject: [PATCH 05/34] wip --- .../api/rest/services/HdfsService.scala | 60 ++++--- .../api/rest/services/KafkaService.scala | 53 +------ .../ResolvedJobDefinitionService.scala | 132 ++++++++++++--- .../api/rest/services/HdfsServiceTest.scala | 66 +++++++- .../api/rest/services/KafkaServiceTest.scala | 117 -------------- .../ResolvedJobDefinitionServiceTest.scala | 150 +++++++++++++++++- 6 files changed, 353 insertions(+), 225 deletions(-) delete mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala index 9af7555d6..08f9258a1 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -31,16 +31,16 @@ import scala.io.Source trait HdfsService { type TopicPartitionOffsets = Map[String, Map[Int, Long]] - def getLatestOffsetFile(params: GetLatestOffsetFileParams): Option[(String, Boolean)] - def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] + def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets + def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] def getLatestCommitBatchId(checkpointDir: String): Option[Long] def getLatestOffsetBatchId(checkpointDir: String): Option[Long] } -class GetLatestOffsetFileParams ( +class HdfsParameters ( val keytab: String, val principal: String, val checkpointLocation: String @@ -50,6 +50,8 @@ class GetLatestOffsetFileParams ( class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformationWrapper) extends HdfsService { private val logger = LoggerFactory.getLogger(this.getClass) private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) + private val offsetsDirName = "offsets" + private val commitsDirName = "commits" private lazy val conf = SparkHadoopUtil.get.conf private lazy val fs = FileSystem.get(conf) @@ -68,21 +70,6 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio } } - override def getLatestOffsetFile(params: GetLatestOffsetFileParams): Option[(String, Boolean)] = { - userGroupInformationWrapper.loginUserFromKeytab(params.principal, params.keytab) - val commitBatchIdOpt = getLatestCommitBatchId(params.checkpointLocation) - val offsetBatchIdOpt = getLatestOffsetBatchId(params.checkpointLocation) - - offsetBatchIdOpt.map { offsetBatchId => - val committed = commitBatchIdOpt match { - case Some(commitBatchId) => offsetBatchId == commitBatchId - case None => false - } - val path = new Path(s"${params.checkpointLocation}/${offsetBatchId}") - (path.toString, committed) - } - } - /** * * @param pathStr path to the file as a string @@ -117,7 +104,7 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio * and org.apache.spark.sql.kafka010.JsonUtils * for details on the assumed format */ - def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets = { + override def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets = { val SERIALIZED_VOID_OFFSET = "-" def parseOffset(value: String): Option[TopicPartitionOffsets] = value match { case SERIALIZED_VOID_OFFSET => None @@ -137,16 +124,41 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio .head } - def getLatestCommitBatchId(checkpointDir: String): Option[Long] = { - val commitsDir = new Path(s"$checkpointDir/commits") - getLatestBatchId(commitsDir) + /** + * + * @return an Option of a String, Boolean pair. The string contains the path to the latest offset file, while the + * boolean is true if the offset is committed (i.e. a corresponding commit file exists), and false otherwise. + * None is returned if the offset file does not exist. If the offset file does not exist, the corresponding + * commit file is assumed to also not exist. + */ + override def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] = { + userGroupInformationWrapper.loginUserFromKeytab(params.principal, params.keytab) + val offsetBatchIdOpt = getLatestOffsetBatchId(params.checkpointLocation) + val offsetFilePath = offsetBatchIdOpt.map { offsetBatchId => + val commitBatchIdOpt = getLatestCommitBatchId(params.checkpointLocation) + val committed = commitBatchIdOpt match { + case Some(commitBatchId) => offsetBatchId == commitBatchId + case None => false + } + val path = new Path(s"${params.checkpointLocation}/${offsetsDirName}/${offsetBatchId}") + (path.toString, committed) + } + if (offsetFilePath.isEmpty) { + logger.debug(s"No offset files exist under checkpoint location ${params.checkpointLocation}") + } + offsetFilePath } - def getLatestOffsetBatchId(checkpointDir: String): Option[Long] = { - val commitsDir = new Path(s"$checkpointDir/offsets") + override def getLatestCommitBatchId(checkpointDir: String): Option[Long] = { + val commitsDir = new Path(s"$checkpointDir/$commitsDirName") getLatestBatchId(commitsDir) } + override def getLatestOffsetBatchId(checkpointDir: String): Option[Long] = { + val offsetsDir = new Path(s"$checkpointDir/$offsetsDirName") + getLatestBatchId(offsetsDir) + } + private def getLatestBatchId(path: Path): Option[Long] = { if (fs.exists(path)) { fs.listStatus(path, batchFilesFilter).map { diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index fa2fa9a3c..ba5ada82a 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -20,67 +20,24 @@ import org.apache.kafka.common.TopicPartition import org.slf4j.LoggerFactory import org.springframework.stereotype.Service import za.co.absa.hyperdrive.trigger.configuration.application.{GeneralConfig, KafkaConfig} -import za.co.absa.hyperdrive.trigger.models.enums.JobTypes -import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanceParameters} import java.util.Properties import javax.inject.Inject import scala.collection.mutable trait KafkaService { - def getKafkaProperties(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] } +class KafkaConsumerParameters ( + val topic: String, + val properties: Properties +) + @Service class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) extends KafkaService { private val logger = LoggerFactory.getLogger(this.getClass) private val kafkaConsumersCache: mutable.Map[Properties, KafkaConsumer[String, String]] = mutable.Map() - private val HyperdriveKafkaTopicKey = "reader.kafka.topic" - private val HyperdriveKafkaBrokersKey = "reader.kafka.brokers" - private val HyperdriveKafkaExtraOptionsKey = "reader.option.kafka" - - override def getKafkaProperties(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] = { - if ( - jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || - !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] - ) { - None - } else { - val args = jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters].appArguments - val topicOpt = args - .find(_.startsWith(s"$HyperdriveKafkaTopicKey=")) - .map(_.replace(s"$HyperdriveKafkaTopicKey=", "")) - val kafkaBrokersOpt = args - .find(_.startsWith(s"$HyperdriveKafkaBrokersKey=")) - .map(_.replace(s"$HyperdriveKafkaBrokersKey=", "")) - val kafkaArgs = args - .filter(_.startsWith(s"$HyperdriveKafkaExtraOptionsKey.")) - .map(_.replace(s"$HyperdriveKafkaExtraOptionsKey.", "")) - .filter(_.contains("=")) - .map { s => - val keyValue = s.split("=", 2) - val key = keyValue(0).trim - val value = keyValue(1).trim - (key, value) - } - .toMap - val properties = new Properties() - kafkaArgs.foreach { case (key, value) => properties.setProperty(key, value) } - - (topicOpt, kafkaBrokersOpt) match { - case (Some(topic), Some(kafkaBrokers)) => - properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaBrokers) - Some(topic, properties) - case (_, _) => - logger.warn( - s"Topic and/or bootstrap servers could not be found in app arguments for jobDefinition" + - s" ${jobDefinition.name}. Topic: ${topicOpt}, Bootstrap servers: ${kafkaBrokersOpt}" - ) - None - } - } - } override def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] = { val groupId = s"${kafkaConfig.groupIdPrefix}-${generalConfig.appUniqueId}-getEndOffsets" diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala index b143b3493..834e268f4 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala @@ -16,56 +16,140 @@ package za.co.absa.hyperdrive.trigger.api.rest.services +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.slf4j.LoggerFactory import org.springframework.stereotype.Service import za.co.absa.hyperdrive.driver.drivers.CommandLineIngestionDriver import za.co.absa.hyperdrive.trigger.configuration.application.SparkConfig import za.co.absa.hyperdrive.trigger.models.enums.JobTypes import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanceParameters} +import java.util.Properties import javax.inject.Inject trait ResolvedJobDefinitionService { - def resolveAppArguments(jobDefinition: ResolvedJobDefinition): ResolvedJobDefinition - def getLatestOffsetFileParams(jobDefinition: ResolvedJobDefinition): Option[GetLatestOffsetFileParams] + def getResolvedAppArguments(jobDefinition: ResolvedJobDefinition): Option[Map[String, String]] + def getHdfsParameters(resolvedAppArguments: Map[String, String]): Option[HdfsParameters] + def getKafkaParameters(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] + def isNewJobInstanceRequired(jobDefinition: ResolvedJobDefinition): Boolean } @Service -class ResolvedJobDefinitionServiceImpl @Inject()(sparkConfig: SparkConfig) extends ResolvedJobDefinitionService { +class ResolvedJobDefinitionServiceImpl @Inject()( + sparkConfig: SparkConfig, + hdfsService: HdfsService, + kafkaService: KafkaService) extends ResolvedJobDefinitionService { + private val logger = LoggerFactory.getLogger(this.getClass) private val HyperdriveCheckpointKey = "writer.common.checkpoint.location" + private val HyperdriveKafkaTopicKey = "reader.kafka.topic" + private val HyperdriveKafkaBrokersKey = "reader.kafka.brokers" + private val HyperdriveKafkaExtraOptionsKey = "reader.option.kafka" - override def resolveAppArguments(jobDefinition: ResolvedJobDefinition): ResolvedJobDefinition = { - if ( - jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || - !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] - ) { - jobDefinition + override def getResolvedAppArguments(jobDefinition: ResolvedJobDefinition): Option[Map[String, String]] = { + if (isHyperdriveJob(jobDefinition)) { + logger.warn(s"Job Definition ${jobDefinition} is not a Hyperdrive Job!") + None } else { val jobParameters = jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters] val args = jobParameters.appArguments val config = CommandLineIngestionDriver.parseConfiguration(args.toArray) import scala.collection.JavaConverters._ val resolvedArgs = config.getKeys.asScala.map { - k => s"${k}=${config.getString(k)}" - }.toList + k => k -> config.getString(k) + }.toMap + Some(resolvedArgs) + } + } + + override def getHdfsParameters(resolvedAppArguments: Map[String, String]): Option[HdfsParameters] = { + val hdfsParameters = for { + keytab <- sparkConfig.yarn.additionalConfs.get("spark.yarn.keytab") + principal <- sparkConfig.yarn.additionalConfs.get("spark.yarn.principal") + checkpointLocation <- resolvedAppArguments.get(HyperdriveCheckpointKey) + } yield new HdfsParameters(keytab, principal, checkpointLocation) - jobDefinition.copy(jobParameters = jobParameters.copy(appArguments = resolvedArgs)) + if (hdfsParameters.isEmpty) { + logger.warn(s"Could not extract hdfs parameters from spark config ${sparkConfig}" + + s" and resolved app arguments ${resolvedAppArguments}") } + + hdfsParameters } - override def getLatestOffsetFileParams(jobDefinition: ResolvedJobDefinition): Option[GetLatestOffsetFileParams] = { - if ( - jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || - !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] - ) { + override def getKafkaParameters(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] = { + if (isHyperdriveJob(jobDefinition)) { + logger.warn(s"Job Definition ${jobDefinition} is not a Hyperdrive Job!") None } else { - for { - keytab <- sparkConfig.yarn.additionalConfs.get("spark.yarn.keytab") - principal <- sparkConfig.yarn.additionalConfs.get("spark.yarn.principal") - checkpointLocation <- jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters].appArguments - .find(_.startsWith(HyperdriveCheckpointKey)) - .map(_.replace(s"$HyperdriveCheckpointKey=", "")) - } yield new GetLatestOffsetFileParams(keytab, principal, checkpointLocation) + val args = jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters].appArguments + val kafkaParameters = for { + topic <- args + .find(_.startsWith(s"$HyperdriveKafkaTopicKey=")) + .map(_.replace(s"$HyperdriveKafkaTopicKey=", "")) + brokers <- args + .find(_.startsWith(s"$HyperdriveKafkaBrokersKey=")) + .map(_.replace(s"$HyperdriveKafkaBrokersKey=", "")) + extraArgs = args + .filter(_.startsWith(s"$HyperdriveKafkaExtraOptionsKey.")) + .map(_.replace(s"$HyperdriveKafkaExtraOptionsKey.", "")) + .filter(_.contains("=")) + .map { s => + val keyValue = s.split("=", 2) + val key = keyValue(0).trim + val value = keyValue(1).trim + (key, value) + } + .toMap + } yield { + val properties = new Properties() + properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) + extraArgs.foreach { case (key, value) => properties.setProperty(key, value) } + (topic, properties) + } + + if (kafkaParameters.isEmpty) { + logger.warn(s"Could not find required kafka parameters in job definition ${jobDefinition.name} with args ${args}") + } + kafkaParameters + } + } + + private def isHyperdriveJob(jobDefinition: ResolvedJobDefinition) = + jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || + !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] + + override def isNewJobInstanceRequired(jobDefinition: ResolvedJobDefinition): Boolean = { + val latestOffsetFilePath = for { + resolvedAppArguments <- getResolvedAppArguments(jobDefinition) + hdfsParameters <- getHdfsParameters(resolvedAppArguments) + latestOffsetFilePath <- hdfsService.getLatestOffsetFilePath(hdfsParameters) + } yield latestOffsetFilePath + + if (latestOffsetFilePath.isEmpty || !latestOffsetFilePath.get._2) { + logger.debug(s"New job instance required because offset does not exist or is not committed ${latestOffsetFilePath}") + true + } else { + val allKafkaOffsetsConsumedOpt = for { + kafkaParameters <- getKafkaParameters(jobDefinition) + hdfsAllOffsets <- hdfsService.parseFileAndClose(latestOffsetFilePath.get._1, hdfsService.parseKafkaOffsetStream) + hdfsOffsets <- hdfsAllOffsets.get(kafkaParameters._1) match { + case Some(v) => Some(v) + case None => + logger.warn(s"Could not find offsets for topic ${kafkaParameters._1} in hdfs offsets ${hdfsAllOffsets}") + None + } + kafkaOffsets = kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) + } yield { + val equalKeys = kafkaOffsets.keySet.intersect(hdfsOffsets.keySet).isEmpty + equalKeys && kafkaOffsets.forall { + case (partition, kafkaPartitionOffset) => hdfsOffsets(partition) == kafkaPartitionOffset + } + } + + allKafkaOffsetsConsumedOpt match { + case Some(allKafkaOffsetsConsumed) => !allKafkaOffsetsConsumed + case None => true + } } } } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala index 6f4d23b34..0ae0a29f4 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala @@ -86,10 +86,57 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo result.getMessage shouldBe "Incomplete log file" } + "getLatestOffsetFile" should "get the latest offset file, and it is committed" in { + val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") + createOffsetFile(tmpCheckpointDir, 12) + createCommitFile(tmpCheckpointDir, 12) + + val params = new HdfsParameters( + keytab = "", + principal = "", + checkpointLocation = tmpCheckpointDir.toAbsolutePath.toString + ) + + val result = underTest.getLatestOffsetFilePath(params) + + result.isDefined shouldBe true + result.get._1 shouldBe s"${tmpCheckpointDir.toAbsolutePath.toString}/offsets/12" + result.get._2 shouldBe true + } + + it should "get the latest offset file, and it is not committed" in { + val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") + createOffsetFile(tmpCheckpointDir, 12) + createCommitFile(tmpCheckpointDir, 11) + + val params = new HdfsParameters( + keytab = "", + principal = "", + checkpointLocation = tmpCheckpointDir.toAbsolutePath.toString + ) + + val result = underTest.getLatestOffsetFilePath(params) + + result.isDefined shouldBe true + result.get._1 shouldBe s"${tmpCheckpointDir.toAbsolutePath.toString}/offsets/12" + result.get._2 shouldBe false + } + + it should "return None if the offset file does not exist" in { + val params = new HdfsParameters( + keytab = "", + principal = "", + checkpointLocation = "non-existent" + ) + + val result = underTest.getLatestOffsetFilePath(params) + + result.isDefined shouldBe false + } + "getLatestCommitBatchId" should "get the latest batch id" in { val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - val tmpCommitsDir = Files.createDirectory(tmpCheckpointDir.resolve( "commits")) - (0 to 12).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) + createCommitFile(tmpCheckpointDir, 12) val result = underTest.getLatestCommitBatchId(tmpCheckpointDir.toAbsolutePath.toString) @@ -105,7 +152,7 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo it should "return None if the commits folder is empty" in { val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - val tmpCommitsDir = Files.createDirectory(tmpCheckpointDir.resolve( "commits")) + Files.createDirectory(tmpCheckpointDir.resolve( "commits")) val result = underTest.getLatestCommitBatchId(tmpCheckpointDir.toAbsolutePath.toString) @@ -114,12 +161,21 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo "getLatestOffsetBatchId" should "get the latest batch id" in { val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - val tmpCommitsDir = Files.createDirectory(tmpCheckpointDir.resolve( "offsets")) - (0 to 7).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) + createOffsetFile(tmpCheckpointDir, 7) val result = underTest.getLatestOffsetBatchId(tmpCheckpointDir.toAbsolutePath.toString) result.isDefined shouldBe true result.get shouldBe 7 } + + private def createOffsetFile(checkpointDir: Path, batchId: Int) = { + val tmpCommitsDir = Files.createDirectory(checkpointDir.resolve( "offsets")) + (0 to batchId).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) + } + + private def createCommitFile(checkpointDir: Path, batchId: Int) = { + val tmpCommitsDir = Files.createDirectory(checkpointDir.resolve( "commits")) + (0 to batchId).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) + } } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala deleted file mode 100644 index 0ecd89be4..000000000 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala +++ /dev/null @@ -1,117 +0,0 @@ - -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.hyperdrive.trigger.api.rest.services - -import org.scalatest.{FlatSpec, Matchers} -import za.co.absa.hyperdrive.trigger.configuration.application.{TestGeneralConfig, TestKafkaConfig} -import za.co.absa.hyperdrive.trigger.models.enums.JobTypes -import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, ShellInstanceParameters, SparkInstanceParameters} - -class KafkaServiceTest extends FlatSpec with Matchers { - val underTest = new KafkaServiceImpl(TestKafkaConfig(), TestGeneralConfig()) - - "getKafkaProperties" should "get kafka properties from a resolved job definition" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=some-topic", - "reader.kafka.brokers=http://localhost:9092", - "reader.option.kafka.security.protocol=PLAINTEXT" - ) - ), - order = 1 - ) - - val result = underTest.getKafkaProperties(jobDefinition) - - result.isDefined shouldBe true - result.get._1 shouldBe "some-topic" - val properties = result.get._2 - properties.getProperty("bootstrap.servers") shouldBe "http://localhost:9092" - properties.getProperty("security.protocol") shouldBe "PLAINTEXT" - } - - it should "return None if the topic is not in the app arguments" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.brokers=http://localhost:9092") - ), - order = 1 - ) - - val result = underTest.getKafkaProperties(jobDefinition) - - result shouldBe None - } - - it should "return None if the kafka brokers are not in the app arguments" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.topic=some-topic") - ), - order = 1 - ) - - val result = underTest.getKafkaProperties(jobDefinition) - - result shouldBe None - } - - it should "return None if the job type is not Hyperdrive" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Spark, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.topic=some-topic", "reader.kafka.brokers=http://localhost:9092") - ), - order = 1 - ) - - val result = underTest.getKafkaProperties(jobDefinition) - - result shouldBe None - } - - it should "return None if the jobParameters are not SparkInstanceParameters" in { - val jobDefinition = ResolvedJobDefinition( - "inconsistentJob", - ShellInstanceParameters( - jobType = JobTypes.Hyperdrive, - scriptLocation = "script.sh" - ), - order = 1 - ) - - val result = underTest.getKafkaProperties(jobDefinition) - - result shouldBe None - } -} diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala index 1613dbfd9..c852af476 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala @@ -16,15 +16,19 @@ package za.co.absa.hyperdrive.trigger.api.rest.services +import org.scalatest.mockito.MockitoSugar import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} +import za.co.absa.hyperdrive.trigger.configuration.application.DefaultTestSparkConfig import za.co.absa.hyperdrive.trigger.models.enums.JobTypes -import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanceParameters} +import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, ShellInstanceParameters, SparkInstanceParameters} -class ResolvedJobDefinitionServiceTest extends FlatSpec with Matchers with BeforeAndAfter { - private val underTest = new ResolvedJobDefinitionServiceImpl() +class ResolvedJobDefinitionServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { + private val hdfsService = mock[HdfsService] + private val kafkaService = mock[KafkaService] + private val underTest = new ResolvedJobDefinitionServiceImpl(DefaultTestSparkConfig().yarn, hdfsService, kafkaService) - "resolveAppArguments" should "successfully parse a file" in { + "getResolvedAppArguments" should "return the resolved app arguments" in { val jobDefinition = ResolvedJobDefinition( "job", SparkInstanceParameters( @@ -38,10 +42,142 @@ class ResolvedJobDefinitionServiceTest extends FlatSpec with Matchers with Befor ), order = 1 ) - val result = underTest.resolveAppArguments(jobDefinition) + val result = underTest.getResolvedAppArguments(jobDefinition) - result.jobParameters.asInstanceOf[SparkInstanceParameters].appArguments should contain theSameElementsAs - Seq("reader.kafka.topic=my-topic", "writer.common.checkpoint.location=/checkpoint/path/my-topic") + result.isDefined shouldBe true + result.get should contain theSameElementsAs Map( + "reader.kafka.topic" -> "my-topic", + "writer.common.checkpoint.location" -> "/checkpoint/path/my-topic" + ) + } + + it should "return None if the job type is not hyperdrive" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Spark, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=my-topic", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + + val result = underTest.getResolvedAppArguments(jobDefinition) + + result shouldBe None + } + + "getHdfsParameters" should "get the hdfs parameters from the configs" in { + val config = DefaultTestSparkConfig().copy(additionalConfs = Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + )) + val underTest = new ResolvedJobDefinitionServiceImpl(config.yarn, hdfsService, kafkaService) + val appArguments = Map( + "writer.common.checkpoint.location" -> "/checkpoint/path/my-topic" + ) + + val params = underTest.getHdfsParameters(appArguments) + + params.isDefined shouldBe true + params.get.keytab shouldBe "/path/to/keytab" + params.get.principal shouldBe "principal" + params.get.checkpointLocation shouldBe "/checkpoint/path/my-topic" + } + + + "getKafkaParameters" should "get kafka properties from a resolved job definition" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "reader.option.kafka.security.protocol=PLAINTEXT" + ) + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result.isDefined shouldBe true + result.get._1 shouldBe "some-topic" + val properties = result.get._2 + properties.getProperty("bootstrap.servers") shouldBe "http://localhost:9092" + properties.getProperty("security.protocol") shouldBe "PLAINTEXT" + } + + it should "return None if the topic is not in the app arguments" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.brokers=http://localhost:9092") + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result shouldBe None + } + + it should "return None if the kafka brokers are not in the app arguments" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.topic=some-topic") + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result shouldBe None } + it should "return None if the job type is not Hyperdrive" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Spark, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.topic=some-topic", "reader.kafka.brokers=http://localhost:9092") + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result shouldBe None + } + + it should "return None if the jobParameters are not SparkInstanceParameters" in { + val jobDefinition = ResolvedJobDefinition( + "inconsistentJob", + ShellInstanceParameters( + jobType = JobTypes.Hyperdrive, + scriptLocation = "script.sh" + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result shouldBe None + } } From 94c0731d4e13aa4329ed2ffc37341c7e2382ed64 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 22 Jun 2022 17:32:34 +0200 Subject: [PATCH 06/34] rename to HyperdriveOffsetComparisonService, add tests --- ...> HyperdriveOffsetComparisonService.scala} | 8 +- ...yperdriveOffsetComparisonServiceTest.scala | 413 ++++++++++++++++++ .../ResolvedJobDefinitionServiceTest.scala | 183 -------- 3 files changed, 417 insertions(+), 187 deletions(-) rename src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/{ResolvedJobDefinitionService.scala => HyperdriveOffsetComparisonService.scala} (96%) create mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala delete mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala similarity index 96% rename from src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala rename to src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 834e268f4..1572eb8ea 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -27,7 +27,7 @@ import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanc import java.util.Properties import javax.inject.Inject -trait ResolvedJobDefinitionService { +trait HyperdriveOffsetComparisonService { def getResolvedAppArguments(jobDefinition: ResolvedJobDefinition): Option[Map[String, String]] def getHdfsParameters(resolvedAppArguments: Map[String, String]): Option[HdfsParameters] def getKafkaParameters(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] @@ -35,10 +35,10 @@ trait ResolvedJobDefinitionService { } @Service -class ResolvedJobDefinitionServiceImpl @Inject()( +class HyperdriveOffsetComparisonServiceImpl @Inject()( sparkConfig: SparkConfig, hdfsService: HdfsService, - kafkaService: KafkaService) extends ResolvedJobDefinitionService { + kafkaService: KafkaService) extends HyperdriveOffsetComparisonService { private val logger = LoggerFactory.getLogger(this.getClass) private val HyperdriveCheckpointKey = "writer.common.checkpoint.location" private val HyperdriveKafkaTopicKey = "reader.kafka.topic" @@ -140,7 +140,7 @@ class ResolvedJobDefinitionServiceImpl @Inject()( } kafkaOffsets = kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) } yield { - val equalKeys = kafkaOffsets.keySet.intersect(hdfsOffsets.keySet).isEmpty + val equalKeys = kafkaOffsets.keySet == hdfsOffsets.keySet equalKeys && kafkaOffsets.forall { case (partition, kafkaPartitionOffset) => hdfsOffsets(partition) == kafkaPartitionOffset } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala new file mode 100644 index 000000000..a2acf7772 --- /dev/null +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -0,0 +1,413 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.mockito.ArgumentCaptor +import org.mockito.ArgumentMatchers.{any, eq => eqTo} +import org.mockito.Mockito.{never, reset, verify, when} +import org.scalatest.mockito.MockitoSugar +import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} +import za.co.absa.hyperdrive.trigger.configuration.application.DefaultTestSparkConfig +import za.co.absa.hyperdrive.trigger.models.enums.JobTypes +import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, ShellInstanceParameters, SparkInstanceParameters} + + +class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { + private val hdfsService = mock[HdfsService] + private val kafkaService = mock[KafkaService] + private val underTest = new HyperdriveOffsetComparisonServiceImpl(DefaultTestSparkConfig().yarn, hdfsService, kafkaService) + + before { + reset(hdfsService) + reset(kafkaService) + } + + "getResolvedAppArguments" should "return the resolved app arguments" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=my-topic", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + val result = underTest.getResolvedAppArguments(jobDefinition) + + result.isDefined shouldBe true + result.get should contain theSameElementsAs Map( + "reader.kafka.topic" -> "my-topic", + "writer.common.checkpoint.location" -> "/checkpoint/path/my-topic" + ) + } + + it should "return None if the job type is not hyperdrive" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Spark, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=my-topic", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + + val result = underTest.getResolvedAppArguments(jobDefinition) + + result shouldBe None + } + + "getHdfsParameters" should "get the hdfs parameters from the configs" in { + val config = DefaultTestSparkConfig().copy(additionalConfs = Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + )) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val appArguments = Map( + "writer.common.checkpoint.location" -> "/checkpoint/path/my-topic" + ) + + val params = underTest.getHdfsParameters(appArguments) + + params.isDefined shouldBe true + params.get.keytab shouldBe "/path/to/keytab" + params.get.principal shouldBe "principal" + params.get.checkpointLocation shouldBe "/checkpoint/path/my-topic" + } + + + "getKafkaParameters" should "get kafka properties from a resolved job definition" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "reader.option.kafka.security.protocol=PLAINTEXT" + ) + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result.isDefined shouldBe true + result.get._1 shouldBe "some-topic" + val properties = result.get._2 + properties.getProperty("bootstrap.servers") shouldBe "http://localhost:9092" + properties.getProperty("security.protocol") shouldBe "PLAINTEXT" + } + + it should "return None if the topic is not in the app arguments" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.brokers=http://localhost:9092") + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result shouldBe None + } + + it should "return None if the kafka brokers are not in the app arguments" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.topic=some-topic") + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result shouldBe None + } + + it should "return None if the job type is not Hyperdrive" in { + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Spark, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.topic=some-topic", "reader.kafka.brokers=http://localhost:9092") + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result shouldBe None + } + + it should "return None if the jobParameters are not SparkInstanceParameters" in { + val jobDefinition = ResolvedJobDefinition( + "inconsistentJob", + ShellInstanceParameters( + jobType = JobTypes.Hyperdrive, + scriptLocation = "script.sh" + ), + order = 1 + ) + + val result = underTest.getKafkaParameters(jobDefinition) + + result shouldBe None + } + + "isNewJobInstanceRequired" should "return false if the kafka and checkpoint folder offsets are the same" in { + val config = DefaultTestSparkConfig().copy(additionalConfs = Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + )) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + + when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("/checkpoint/path/some-topic/offsets/21", true))) + when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + .thenReturn(Option(Map("some-topic" -> Map(2 -> 2021L, 0 -> 21L, 1 -> 1021L)))) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) + + val result = underTest.isNewJobInstanceRequired(jobDefinition) + + val hdfsParametersCaptor: ArgumentCaptor[HdfsParameters] = ArgumentCaptor.forClass(classOf[HdfsParameters]) + verify(hdfsService).getLatestOffsetFilePath(hdfsParametersCaptor.capture()) + hdfsParametersCaptor.getValue.keytab shouldBe "/path/to/keytab" + hdfsParametersCaptor.getValue.principal shouldBe "principal" + hdfsParametersCaptor.getValue.checkpointLocation shouldBe "/checkpoint/path/some-topic" + verify(hdfsService).parseFileAndClose(eqTo("/checkpoint/path/some-topic/offsets/21"), any()) + verify(kafkaService).getEndOffsets(eqTo("some-topic"), any()) + result shouldBe false + } + + it should "return true if no offset file is present" in { + val config = DefaultTestSparkConfig().copy(additionalConfs = Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + )) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(None) + + val result = underTest.isNewJobInstanceRequired(jobDefinition) + + verify(hdfsService).getLatestOffsetFilePath(any()) + result shouldBe true + } + + + it should "return true if the offset is not committed" in { + val config = DefaultTestSparkConfig().copy(additionalConfs = Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + )) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", false))) + + val result = underTest.isNewJobInstanceRequired(jobDefinition) + + verify(hdfsService).getLatestOffsetFilePath(any()) + result shouldBe true + } + + it should "return true if a offset file could not be read" in { + val config = DefaultTestSparkConfig().copy(additionalConfs = Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + )) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + + when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) + when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())).thenReturn(None) + + val result = underTest.isNewJobInstanceRequired(jobDefinition) + + verify(hdfsService).getLatestOffsetFilePath(any()) + verify(hdfsService).parseFileAndClose(any(), any()) + result shouldBe true + } + + it should "return true if the checkpoints offset does not contain the topic" in { + val config = DefaultTestSparkConfig().copy(additionalConfs = Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + )) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) + when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + .thenReturn(Option(Map("some-other-topic" -> Map(0 -> 21L)))) + + val result = underTest.isNewJobInstanceRequired(jobDefinition) + + verify(hdfsService).getLatestOffsetFilePath(any()) + verify(hdfsService).parseFileAndClose(any(), any()) + verify(kafkaService, never()).getEndOffsets(any(), any()) + result shouldBe true + } + + it should "return true if the kafka offsets and checkpoint offset do not have the same set of partitions" in { + val config = DefaultTestSparkConfig().copy(additionalConfs = Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + )) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + + when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) + when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + .thenReturn(Option(Map("some-topic" -> Map(0 -> 21L)))) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) + + val result = underTest.isNewJobInstanceRequired(jobDefinition) + + verify(hdfsService).getLatestOffsetFilePath(any()) + verify(hdfsService).parseFileAndClose(any(), any()) + verify(kafkaService).getEndOffsets(any(), any()) + result shouldBe true + } + + it should "return true if the kafka offsets and checkpoint offsets are not the same" in { + val config = DefaultTestSparkConfig().copy(additionalConfs = Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + )) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = ResolvedJobDefinition( + "job", + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) + ), + order = 1 + ) + + when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) + when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + .thenReturn(Option(Map("some-topic" -> Map(0 -> 42L, 1 -> 55L)))) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 42L, 1 -> 7L)) + + val result = underTest.isNewJobInstanceRequired(jobDefinition) + + verify(hdfsService).getLatestOffsetFilePath(any()) + verify(hdfsService).parseFileAndClose(any(), any()) + verify(kafkaService).getEndOffsets(any(), any()) + result shouldBe true + } +} diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala deleted file mode 100644 index c852af476..000000000 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/ResolvedJobDefinitionServiceTest.scala +++ /dev/null @@ -1,183 +0,0 @@ - -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.hyperdrive.trigger.api.rest.services - -import org.scalatest.mockito.MockitoSugar -import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} -import za.co.absa.hyperdrive.trigger.configuration.application.DefaultTestSparkConfig -import za.co.absa.hyperdrive.trigger.models.enums.JobTypes -import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, ShellInstanceParameters, SparkInstanceParameters} - - -class ResolvedJobDefinitionServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { - private val hdfsService = mock[HdfsService] - private val kafkaService = mock[KafkaService] - private val underTest = new ResolvedJobDefinitionServiceImpl(DefaultTestSparkConfig().yarn, hdfsService, kafkaService) - - "getResolvedAppArguments" should "return the resolved app arguments" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=my-topic", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 - ) - val result = underTest.getResolvedAppArguments(jobDefinition) - - result.isDefined shouldBe true - result.get should contain theSameElementsAs Map( - "reader.kafka.topic" -> "my-topic", - "writer.common.checkpoint.location" -> "/checkpoint/path/my-topic" - ) - } - - it should "return None if the job type is not hyperdrive" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Spark, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=my-topic", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 - ) - - val result = underTest.getResolvedAppArguments(jobDefinition) - - result shouldBe None - } - - "getHdfsParameters" should "get the hdfs parameters from the configs" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - )) - val underTest = new ResolvedJobDefinitionServiceImpl(config.yarn, hdfsService, kafkaService) - val appArguments = Map( - "writer.common.checkpoint.location" -> "/checkpoint/path/my-topic" - ) - - val params = underTest.getHdfsParameters(appArguments) - - params.isDefined shouldBe true - params.get.keytab shouldBe "/path/to/keytab" - params.get.principal shouldBe "principal" - params.get.checkpointLocation shouldBe "/checkpoint/path/my-topic" - } - - - "getKafkaParameters" should "get kafka properties from a resolved job definition" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=some-topic", - "reader.kafka.brokers=http://localhost:9092", - "reader.option.kafka.security.protocol=PLAINTEXT" - ) - ), - order = 1 - ) - - val result = underTest.getKafkaParameters(jobDefinition) - - result.isDefined shouldBe true - result.get._1 shouldBe "some-topic" - val properties = result.get._2 - properties.getProperty("bootstrap.servers") shouldBe "http://localhost:9092" - properties.getProperty("security.protocol") shouldBe "PLAINTEXT" - } - - it should "return None if the topic is not in the app arguments" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.brokers=http://localhost:9092") - ), - order = 1 - ) - - val result = underTest.getKafkaParameters(jobDefinition) - - result shouldBe None - } - - it should "return None if the kafka brokers are not in the app arguments" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.topic=some-topic") - ), - order = 1 - ) - - val result = underTest.getKafkaParameters(jobDefinition) - - result shouldBe None - } - - it should "return None if the job type is not Hyperdrive" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Spark, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.topic=some-topic", "reader.kafka.brokers=http://localhost:9092") - ), - order = 1 - ) - - val result = underTest.getKafkaParameters(jobDefinition) - - result shouldBe None - } - - it should "return None if the jobParameters are not SparkInstanceParameters" in { - val jobDefinition = ResolvedJobDefinition( - "inconsistentJob", - ShellInstanceParameters( - jobType = JobTypes.Hyperdrive, - scriptLocation = "script.sh" - ), - order = 1 - ) - - val result = underTest.getKafkaParameters(jobDefinition) - - result shouldBe None - } -} From 3a4e8d75b1b794a90990c2eb7a3f46d1025a28ae Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 22 Jun 2022 17:47:56 +0200 Subject: [PATCH 07/34] fix tests --- .../trigger/api/rest/services/HdfsService.scala | 3 +-- .../services/HyperdriveOffsetComparisonService.scala | 12 +++++++++--- .../trigger/api/rest/services/HdfsServiceTest.scala | 9 +++++++++ .../HyperdriveOffsetComparisonServiceTest.scala | 5 +++-- 4 files changed, 22 insertions(+), 7 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala index 08f9258a1..7c0f4f5b7 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -88,8 +88,7 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio } catch { case e: Exception => // re-throw the exception with the log file path added - throw new Exception( - s"Failed to parse file $path. ${e.getMessage}", e) + throw new Exception(s"Failed to parse file $path", e) } finally { IOUtils.closeQuietly(input) } diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 1572eb8ea..92e8a195d 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -26,6 +26,7 @@ import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanc import java.util.Properties import javax.inject.Inject +import scala.util.Try trait HyperdriveOffsetComparisonService { def getResolvedAppArguments(jobDefinition: ResolvedJobDefinition): Option[Map[String, String]] @@ -131,7 +132,12 @@ class HyperdriveOffsetComparisonServiceImpl @Inject()( } else { val allKafkaOffsetsConsumedOpt = for { kafkaParameters <- getKafkaParameters(jobDefinition) - hdfsAllOffsets <- hdfsService.parseFileAndClose(latestOffsetFilePath.get._1, hdfsService.parseKafkaOffsetStream) + hdfsAllOffsets <- Try(hdfsService.parseFileAndClose(latestOffsetFilePath.get._1, hdfsService.parseKafkaOffsetStream)) + .recover { + case e: Exception => + logger.warn(s"Couldn't parse file ${latestOffsetFilePath.get._1}", e) + None + }.toOption.flatten hdfsOffsets <- hdfsAllOffsets.get(kafkaParameters._1) match { case Some(v) => Some(v) case None => @@ -140,8 +146,8 @@ class HyperdriveOffsetComparisonServiceImpl @Inject()( } kafkaOffsets = kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) } yield { - val equalKeys = kafkaOffsets.keySet == hdfsOffsets.keySet - equalKeys && kafkaOffsets.forall { + val isSamePartitions = kafkaOffsets.keySet == hdfsOffsets.keySet + isSamePartitions && kafkaOffsets.forall { case (partition, kafkaPartitionOffset) => hdfsOffsets(partition) == kafkaPartitionOffset } } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala index 0ae0a29f4..86612659e 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala @@ -62,6 +62,15 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo result shouldBe None } + it should "rethrow an exception if parsing throws an error" in { + val tmpFile = Files.createTempFile(baseDirPath, "hdfsServiceTest", "") + val parseFn: Iterator[String] => Seq[Int] = _ => throw new Exception() + + val result = the [Exception] thrownBy underTest.parseFileAndClose(tmpFile.toAbsolutePath.toString, parseFn) + + result.getMessage should include(tmpFile.toAbsolutePath.toString) + } + "parseKafkaOffsetStream" should "parse an offset file" in { val lines = Seq( "v1", diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala index a2acf7772..2a0afa478 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -280,7 +280,7 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with result shouldBe true } - it should "return true if a offset file could not be read" in { + it should "return true if a offset file could not be parsed" in { val config = DefaultTestSparkConfig().copy(additionalConfs = Map( "spark.yarn.keytab" -> "/path/to/keytab", "spark.yarn.principal" -> "principal" @@ -302,7 +302,8 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with ) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())).thenReturn(None) + when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + .thenThrow(new RuntimeException("Failed to parse")) val result = underTest.isNewJobInstanceRequired(jobDefinition) From 845f6f5cbde6eb4d04361d8218902b3f7da2f600 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 22 Jun 2022 17:48:51 +0200 Subject: [PATCH 08/34] fix formatting --- .../api/rest/services/HdfsService.scala | 36 ++++---- .../HyperdriveOffsetComparisonService.scala | 48 +++++----- .../api/rest/services/KafkaService.scala | 2 +- .../api/rest/services/HdfsServiceTest.scala | 10 +-- ...yperdriveOffsetComparisonServiceTest.scala | 87 +++++++++++-------- 5 files changed, 100 insertions(+), 83 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala index 7c0f4f5b7..549c62930 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -1,4 +1,3 @@ - /* * Copyright 2018 ABSA Group Limited * @@ -40,14 +39,14 @@ trait HdfsService { def getLatestOffsetBatchId(checkpointDir: String): Option[Long] } -class HdfsParameters ( +class HdfsParameters( val keytab: String, val principal: String, val checkpointLocation: String ) @Service -class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformationWrapper) extends HdfsService { +class HdfsServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformationWrapper) extends HdfsService { private val logger = LoggerFactory.getLogger(this.getClass) private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) private val offsetsDirName = "offsets" @@ -56,7 +55,7 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio private lazy val fs = FileSystem.get(conf) /** - * See org.apache.spark.sql.execution.streaming.HDFSMetadataLog + * See org.apache.spark.sql.execution.streaming.HDFSMetadataLog */ private val batchFilesFilter = new PathFilter { override def accept(path: Path): Boolean = { @@ -71,12 +70,11 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio } /** - * - * @param pathStr path to the file as a string - * @param parseFn function that parses the file line by line. Caution: It must materialize the content, + * @param pathStr path to the file as a string + * @param parseFn function that parses the file line by line. Caution: It must materialize the content, * because the file is closed after the method completes. E.g. it must not return an iterator. - * @tparam R type of the parsed value - * @return None if the file doesn't exist, Some with the parsed content + * @tparam R type of the parsed value + * @return None if the file doesn't exist, Some with the parsed content */ override def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] = { val path = new Path(pathStr) @@ -99,15 +97,15 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio } /** - * see org.apache.spark.sql.execution.streaming.OffsetSeqLog - * and org.apache.spark.sql.kafka010.JsonUtils - * for details on the assumed format + * see org.apache.spark.sql.execution.streaming.OffsetSeqLog + * and org.apache.spark.sql.kafka010.JsonUtils + * for details on the assumed format */ override def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets = { val SERIALIZED_VOID_OFFSET = "-" def parseOffset(value: String): Option[TopicPartitionOffsets] = value match { case SERIALIZED_VOID_OFFSET => None - case json => Some(mapper.readValue(json, classOf[TopicPartitionOffsets])) + case json => Some(mapper.readValue(json, classOf[TopicPartitionOffsets])) } if (!lines.hasNext) { throw new IllegalStateException("Incomplete log file") @@ -124,8 +122,7 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio } /** - * - * @return an Option of a String, Boolean pair. The string contains the path to the latest offset file, while the + * @return an Option of a String, Boolean pair. The string contains the path to the latest offset file, while the * boolean is true if the offset is committed (i.e. a corresponding commit file exists), and false otherwise. * None is returned if the offset file does not exist. If the offset file does not exist, the corresponding * commit file is assumed to also not exist. @@ -137,7 +134,7 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio val commitBatchIdOpt = getLatestCommitBatchId(params.checkpointLocation) val committed = commitBatchIdOpt match { case Some(commitBatchId) => offsetBatchId == commitBatchId - case None => false + case None => false } val path = new Path(s"${params.checkpointLocation}/${offsetsDirName}/${offsetBatchId}") (path.toString, committed) @@ -160,9 +157,10 @@ class HdfsServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformatio private def getLatestBatchId(path: Path): Option[Long] = { if (fs.exists(path)) { - fs.listStatus(path, batchFilesFilter).map { - status => status.getPath.getName.toLong - } + fs.listStatus(path, batchFilesFilter) + .map { status => + status.getPath.getName.toLong + } .sorted .lastOption } else { diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 92e8a195d..ec19ecf39 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -1,4 +1,3 @@ - /* * Copyright 2018 ABSA Group Limited * @@ -36,10 +35,10 @@ trait HyperdriveOffsetComparisonService { } @Service -class HyperdriveOffsetComparisonServiceImpl @Inject()( - sparkConfig: SparkConfig, - hdfsService: HdfsService, - kafkaService: KafkaService) extends HyperdriveOffsetComparisonService { +class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, + hdfsService: HdfsService, + kafkaService: KafkaService +) extends HyperdriveOffsetComparisonService { private val logger = LoggerFactory.getLogger(this.getClass) private val HyperdriveCheckpointKey = "writer.common.checkpoint.location" private val HyperdriveKafkaTopicKey = "reader.kafka.topic" @@ -55,8 +54,8 @@ class HyperdriveOffsetComparisonServiceImpl @Inject()( val args = jobParameters.appArguments val config = CommandLineIngestionDriver.parseConfiguration(args.toArray) import scala.collection.JavaConverters._ - val resolvedArgs = config.getKeys.asScala.map { - k => k -> config.getString(k) + val resolvedArgs = config.getKeys.asScala.map { k => + k -> config.getString(k) }.toMap Some(resolvedArgs) } @@ -70,8 +69,10 @@ class HyperdriveOffsetComparisonServiceImpl @Inject()( } yield new HdfsParameters(keytab, principal, checkpointLocation) if (hdfsParameters.isEmpty) { - logger.warn(s"Could not extract hdfs parameters from spark config ${sparkConfig}" + - s" and resolved app arguments ${resolvedAppArguments}") + logger.warn( + s"Could not extract hdfs parameters from spark config ${sparkConfig}" + + s" and resolved app arguments ${resolvedAppArguments}" + ) } hdfsParameters @@ -109,7 +110,9 @@ class HyperdriveOffsetComparisonServiceImpl @Inject()( } if (kafkaParameters.isEmpty) { - logger.warn(s"Could not find required kafka parameters in job definition ${jobDefinition.name} with args ${args}") + logger.warn( + s"Could not find required kafka parameters in job definition ${jobDefinition.name} with args ${args}" + ) } kafkaParameters } @@ -127,17 +130,22 @@ class HyperdriveOffsetComparisonServiceImpl @Inject()( } yield latestOffsetFilePath if (latestOffsetFilePath.isEmpty || !latestOffsetFilePath.get._2) { - logger.debug(s"New job instance required because offset does not exist or is not committed ${latestOffsetFilePath}") + logger.debug( + s"New job instance required because offset does not exist or is not committed ${latestOffsetFilePath}" + ) true } else { val allKafkaOffsetsConsumedOpt = for { kafkaParameters <- getKafkaParameters(jobDefinition) - hdfsAllOffsets <- Try(hdfsService.parseFileAndClose(latestOffsetFilePath.get._1, hdfsService.parseKafkaOffsetStream)) - .recover { - case e: Exception => - logger.warn(s"Couldn't parse file ${latestOffsetFilePath.get._1}", e) - None - }.toOption.flatten + hdfsAllOffsets <- Try( + hdfsService.parseFileAndClose(latestOffsetFilePath.get._1, hdfsService.parseKafkaOffsetStream) + ) + .recover { case e: Exception => + logger.warn(s"Couldn't parse file ${latestOffsetFilePath.get._1}", e) + None + } + .toOption + .flatten hdfsOffsets <- hdfsAllOffsets.get(kafkaParameters._1) match { case Some(v) => Some(v) case None => @@ -147,14 +155,14 @@ class HyperdriveOffsetComparisonServiceImpl @Inject()( kafkaOffsets = kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) } yield { val isSamePartitions = kafkaOffsets.keySet == hdfsOffsets.keySet - isSamePartitions && kafkaOffsets.forall { - case (partition, kafkaPartitionOffset) => hdfsOffsets(partition) == kafkaPartitionOffset + isSamePartitions && kafkaOffsets.forall { case (partition, kafkaPartitionOffset) => + hdfsOffsets(partition) == kafkaPartitionOffset } } allKafkaOffsetsConsumedOpt match { case Some(allKafkaOffsetsConsumed) => !allKafkaOffsetsConsumed - case None => true + case None => true } } } diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index ba5ada82a..3e892aefb 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -29,7 +29,7 @@ trait KafkaService { def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] } -class KafkaConsumerParameters ( +class KafkaConsumerParameters( val topic: String, val properties: Properties ) diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala index 86612659e..fe694e947 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala @@ -1,4 +1,3 @@ - /* * Copyright 2018 ABSA Group Limited * @@ -25,7 +24,6 @@ import za.co.absa.commons.io.TempDirectory import java.nio.charset.StandardCharsets import java.nio.file.{Files, Path} - class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { private val ugiWrapper = mock[UserGroupInformationWrapper] doNothing().when(ugiWrapper).loginUserFromKeytab(any(), any()) @@ -66,7 +64,7 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo val tmpFile = Files.createTempFile(baseDirPath, "hdfsServiceTest", "") val parseFn: Iterator[String] => Seq[Int] = _ => throw new Exception() - val result = the [Exception] thrownBy underTest.parseFileAndClose(tmpFile.toAbsolutePath.toString, parseFn) + val result = the[Exception] thrownBy underTest.parseFileAndClose(tmpFile.toAbsolutePath.toString, parseFn) result.getMessage should include(tmpFile.toAbsolutePath.toString) } @@ -161,7 +159,7 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo it should "return None if the commits folder is empty" in { val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - Files.createDirectory(tmpCheckpointDir.resolve( "commits")) + Files.createDirectory(tmpCheckpointDir.resolve("commits")) val result = underTest.getLatestCommitBatchId(tmpCheckpointDir.toAbsolutePath.toString) @@ -179,12 +177,12 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo } private def createOffsetFile(checkpointDir: Path, batchId: Int) = { - val tmpCommitsDir = Files.createDirectory(checkpointDir.resolve( "offsets")) + val tmpCommitsDir = Files.createDirectory(checkpointDir.resolve("offsets")) (0 to batchId).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) } private def createCommitFile(checkpointDir: Path, batchId: Int) = { - val tmpCommitsDir = Files.createDirectory(checkpointDir.resolve( "commits")) + val tmpCommitsDir = Files.createDirectory(checkpointDir.resolve("commits")) (0 to batchId).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) } } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala index 2a0afa478..c94fee5a8 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -1,4 +1,3 @@ - /* * Copyright 2018 ABSA Group Limited * @@ -25,11 +24,11 @@ import za.co.absa.hyperdrive.trigger.configuration.application.DefaultTestSparkC import za.co.absa.hyperdrive.trigger.models.enums.JobTypes import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, ShellInstanceParameters, SparkInstanceParameters} - class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { private val hdfsService = mock[HdfsService] private val kafkaService = mock[KafkaService] - private val underTest = new HyperdriveOffsetComparisonServiceImpl(DefaultTestSparkConfig().yarn, hdfsService, kafkaService) + private val underTest = + new HyperdriveOffsetComparisonServiceImpl(DefaultTestSparkConfig().yarn, hdfsService, kafkaService) before { reset(hdfsService) @@ -80,10 +79,12 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with } "getHdfsParameters" should "get the hdfs parameters from the configs" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - )) + val config = DefaultTestSparkConfig().copy(additionalConfs = + Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + ) + ) val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) val appArguments = Map( "writer.common.checkpoint.location" -> "/checkpoint/path/my-topic" @@ -97,7 +98,6 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with params.get.checkpointLocation shouldBe "/checkpoint/path/my-topic" } - "getKafkaParameters" should "get kafka properties from a resolved job definition" in { val jobDefinition = ResolvedJobDefinition( "job", @@ -190,10 +190,12 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with } "isNewJobInstanceRequired" should "return false if the kafka and checkpoint folder offsets are the same" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - )) + val config = DefaultTestSparkConfig().copy(additionalConfs = + Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + ) + ) val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) val jobDefinition = ResolvedJobDefinition( "job", @@ -228,10 +230,12 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with } it should "return true if no offset file is present" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - )) + val config = DefaultTestSparkConfig().copy(additionalConfs = + Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + ) + ) val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) val jobDefinition = ResolvedJobDefinition( "job", @@ -253,12 +257,13 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with result shouldBe true } - it should "return true if the offset is not committed" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - )) + val config = DefaultTestSparkConfig().copy(additionalConfs = + Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + ) + ) val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) val jobDefinition = ResolvedJobDefinition( "job", @@ -281,10 +286,12 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with } it should "return true if a offset file could not be parsed" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - )) + val config = DefaultTestSparkConfig().copy(additionalConfs = + Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + ) + ) val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) val jobDefinition = ResolvedJobDefinition( "job", @@ -313,10 +320,12 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with } it should "return true if the checkpoints offset does not contain the topic" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - )) + val config = DefaultTestSparkConfig().copy(additionalConfs = + Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + ) + ) val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) val jobDefinition = ResolvedJobDefinition( "job", @@ -345,10 +354,12 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with } it should "return true if the kafka offsets and checkpoint offset do not have the same set of partitions" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - )) + val config = DefaultTestSparkConfig().copy(additionalConfs = + Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + ) + ) val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) val jobDefinition = ResolvedJobDefinition( "job", @@ -379,10 +390,12 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with } it should "return true if the kafka offsets and checkpoint offsets are not the same" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - )) + val config = DefaultTestSparkConfig().copy(additionalConfs = + Map( + "spark.yarn.keytab" -> "/path/to/keytab", + "spark.yarn.principal" -> "principal" + ) + ) val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) val jobDefinition = ResolvedJobDefinition( "job", From e2e7d4d5c82b5a08ab1e8c6c6d35c2934d76b170 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 23 Jun 2022 10:45:51 +0200 Subject: [PATCH 09/34] make methods private, remove unnecessary stuff --- .../api/rest/services/HdfsService.scala | 8 +-- .../api/rest/services/KafkaService.scala | 5 -- .../api/rest/services/HdfsServiceTest.scala | 54 +++++++++---------- 3 files changed, 27 insertions(+), 40 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala index 549c62930..df7e8fa67 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -33,10 +33,6 @@ trait HdfsService { def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] - - def getLatestCommitBatchId(checkpointDir: String): Option[Long] - - def getLatestOffsetBatchId(checkpointDir: String): Option[Long] } class HdfsParameters( @@ -145,12 +141,12 @@ class HdfsServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformati offsetFilePath } - override def getLatestCommitBatchId(checkpointDir: String): Option[Long] = { + def getLatestCommitBatchId(checkpointDir: String): Option[Long] = { val commitsDir = new Path(s"$checkpointDir/$commitsDirName") getLatestBatchId(commitsDir) } - override def getLatestOffsetBatchId(checkpointDir: String): Option[Long] = { + def getLatestOffsetBatchId(checkpointDir: String): Option[Long] = { val offsetsDir = new Path(s"$checkpointDir/$offsetsDirName") getLatestBatchId(offsetsDir) } diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index 3e892aefb..2e1cf153c 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -29,11 +29,6 @@ trait KafkaService { def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] } -class KafkaConsumerParameters( - val topic: String, - val properties: Properties -) - @Service class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) extends KafkaService { private val logger = LoggerFactory.getLogger(this.getClass) diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala index fe694e947..69c9ecaef 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala @@ -111,10 +111,10 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo result.get._2 shouldBe true } - it should "get the latest offset file, and it is not committed" in { + it should "get the latest offset file, and committed = true, if the commits folder is empty" in { val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") createOffsetFile(tmpCheckpointDir, 12) - createCommitFile(tmpCheckpointDir, 11) + Files.createDirectory(tmpCheckpointDir.resolve("commits")) val params = new HdfsParameters( keytab = "", @@ -129,51 +129,47 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo result.get._2 shouldBe false } - it should "return None if the offset file does not exist" in { + it should "get the latest offset file, and it is not committed" in { + val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") + createOffsetFile(tmpCheckpointDir, 12) + createCommitFile(tmpCheckpointDir, 11) + val params = new HdfsParameters( keytab = "", principal = "", - checkpointLocation = "non-existent" + checkpointLocation = tmpCheckpointDir.toAbsolutePath.toString ) val result = underTest.getLatestOffsetFilePath(params) - result.isDefined shouldBe false - } - - "getLatestCommitBatchId" should "get the latest batch id" in { - val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - createCommitFile(tmpCheckpointDir, 12) - - val result = underTest.getLatestCommitBatchId(tmpCheckpointDir.toAbsolutePath.toString) - result.isDefined shouldBe true - result.get shouldBe 12 + result.get._1 shouldBe s"${tmpCheckpointDir.toAbsolutePath.toString}/offsets/12" + result.get._2 shouldBe false } it should "return None if the checkpoints folder does not exist" in { - val result = underTest.getLatestCommitBatchId("non-existent") - - result.isDefined shouldBe false - } - - it should "return None if the commits folder is empty" in { - val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - Files.createDirectory(tmpCheckpointDir.resolve("commits")) + val params = new HdfsParameters( + keytab = "", + principal = "", + checkpointLocation = "non-existent" + ) - val result = underTest.getLatestCommitBatchId(tmpCheckpointDir.toAbsolutePath.toString) + val result = underTest.getLatestOffsetFilePath(params) result.isDefined shouldBe false } - "getLatestOffsetBatchId" should "get the latest batch id" in { - val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - createOffsetFile(tmpCheckpointDir, 7) + it should "return None if the offsets folder is empty" in { + val params = new HdfsParameters( + keytab = "", + principal = "", + checkpointLocation = "checkpoints" + ) + Files.createTempDirectory(baseDirPath, "checkpoints") - val result = underTest.getLatestOffsetBatchId(tmpCheckpointDir.toAbsolutePath.toString) + val result = underTest.getLatestOffsetFilePath(params) - result.isDefined shouldBe true - result.get shouldBe 7 + result.isDefined shouldBe false } private def createOffsetFile(checkpointDir: Path, batchId: Int) = { From 7661d2a3feb0f547402fd36ee8b6a21759077130 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 24 Jun 2022 17:19:06 +0200 Subject: [PATCH 10/34] Don't import hyperdrive -ingestor --- pom.xml | 9 +++--- .../HyperdriveOffsetComparisonService.scala | 29 +++++++++++++++++-- 2 files changed, 31 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index 356a58dc6..d33c371e7 100644 --- a/pom.xml +++ b/pom.xml @@ -75,7 +75,7 @@ 42.3.3 2.2.0 2.2.0 - 4.6.0 + 2.7 2.7.3 2.0.3 2.0.2 @@ -186,11 +186,10 @@ kafka_${scala.compat.version} ${kafka.version} - - za.co.absa.hyperdrive - driver_${scala.compat.version} - ${hyperdrive.version} + org.apache.commons + commons-configuration2 + ${commons.configuration2.version} diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index ec19ecf39..d305376b3 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -15,10 +15,13 @@ package za.co.absa.hyperdrive.trigger.api.rest.services +import org.apache.commons.configuration2.builder.BasicConfigurationBuilder +import org.apache.commons.configuration2.builder.fluent.Parameters +import org.apache.commons.configuration2.convert.DefaultListDelimiterHandler +import org.apache.commons.configuration2.{BaseConfiguration, Configuration} import org.apache.kafka.clients.consumer.ConsumerConfig import org.slf4j.LoggerFactory import org.springframework.stereotype.Service -import za.co.absa.hyperdrive.driver.drivers.CommandLineIngestionDriver import za.co.absa.hyperdrive.trigger.configuration.application.SparkConfig import za.co.absa.hyperdrive.trigger.models.enums.JobTypes import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanceParameters} @@ -44,6 +47,8 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private val HyperdriveKafkaTopicKey = "reader.kafka.topic" private val HyperdriveKafkaBrokersKey = "reader.kafka.brokers" private val HyperdriveKafkaExtraOptionsKey = "reader.option.kafka" + private val PropertyDelimiter = "=" + private val ListDelimiter = ',' override def getResolvedAppArguments(jobDefinition: ResolvedJobDefinition): Option[Map[String, String]] = { if (isHyperdriveJob(jobDefinition)) { @@ -52,7 +57,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } else { val jobParameters = jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters] val args = jobParameters.appArguments - val config = CommandLineIngestionDriver.parseConfiguration(args.toArray) + val config = parseConfiguration(args.toArray) import scala.collection.JavaConverters._ val resolvedArgs = config.getKeys.asScala.map { k => k -> config.getString(k) @@ -166,4 +171,24 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } } } + + private def parseConfiguration(settings: Array[String]): Configuration = { + val configuration = new BasicConfigurationBuilder[BaseConfiguration](classOf[BaseConfiguration]) + .configure(new Parameters() + .basic() + .setListDelimiterHandler(new DefaultListDelimiterHandler(ListDelimiter))) + .getConfiguration + + settings.foreach(setOrThrow(_, configuration)) + configuration + } + + private def setOrThrow(setting: String, configuration: Configuration): Unit = { + if(!setting.contains(PropertyDelimiter)) { + throw new IllegalArgumentException(s"Invalid setting format: $setting") + } else { + val settingKeyValue = setting.split(PropertyDelimiter, 2) + configuration.setProperty(settingKeyValue(0).trim, settingKeyValue(1).trim) + } + } } From 69b04031b863e447ff414e9567e2b61f91b1854c Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Tue, 28 Jun 2022 17:01:20 +0200 Subject: [PATCH 11/34] Exclude conflicting dependeny --- pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pom.xml b/pom.xml index d33c371e7..724a3b79a 100644 --- a/pom.xml +++ b/pom.xml @@ -250,6 +250,10 @@ org.slf4j slf4j-log4j12 + + commons-beanutils + commons-beanutils-core + From 9ce0843d19c56d63c199c776d34f3e3c1bd42f9a Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Tue, 28 Jun 2022 17:06:23 +0200 Subject: [PATCH 12/34] Fix format --- .../services/HyperdriveOffsetComparisonService.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index d305376b3..379efefce 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -174,9 +174,11 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private def parseConfiguration(settings: Array[String]): Configuration = { val configuration = new BasicConfigurationBuilder[BaseConfiguration](classOf[BaseConfiguration]) - .configure(new Parameters() - .basic() - .setListDelimiterHandler(new DefaultListDelimiterHandler(ListDelimiter))) + .configure( + new Parameters() + .basic() + .setListDelimiterHandler(new DefaultListDelimiterHandler(ListDelimiter)) + ) .getConfiguration settings.foreach(setOrThrow(_, configuration)) @@ -184,7 +186,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } private def setOrThrow(setting: String, configuration: Configuration): Unit = { - if(!setting.contains(PropertyDelimiter)) { + if (!setting.contains(PropertyDelimiter)) { throw new IllegalArgumentException(s"Invalid setting format: $setting") } else { val settingKeyValue = setting.split(PropertyDelimiter, 2) From d9ff9e2fbdeff378bcef861f4db546979dacd8f5 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 30 Jun 2022 10:25:47 +0200 Subject: [PATCH 13/34] PR fixes --- .../trigger/api/rest/services/KafkaService.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index 2e1cf153c..79636c1f5 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -17,13 +17,12 @@ package za.co.absa.hyperdrive.trigger.api.rest.services import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.common.TopicPartition -import org.slf4j.LoggerFactory import org.springframework.stereotype.Service import za.co.absa.hyperdrive.trigger.configuration.application.{GeneralConfig, KafkaConfig} import java.util.Properties +import java.util.concurrent.ConcurrentHashMap import javax.inject.Inject -import scala.collection.mutable trait KafkaService { def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] @@ -31,13 +30,14 @@ trait KafkaService { @Service class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) extends KafkaService { - private val logger = LoggerFactory.getLogger(this.getClass) - private val kafkaConsumersCache: mutable.Map[Properties, KafkaConsumer[String, String]] = mutable.Map() + private val kafkaConsumersCache = new ConcurrentHashMap[Properties, KafkaConsumer[String, String]]() override def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] = { val groupId = s"${kafkaConfig.groupIdPrefix}-${generalConfig.appUniqueId}-getEndOffsets" consumerProperties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + import scala.collection.JavaConverters._ val consumer = kafkaConsumersCache + .asScala .getOrElse(consumerProperties, { val consumer = new KafkaConsumer[String, String](consumerProperties) kafkaConsumersCache.put(consumerProperties, consumer) @@ -45,7 +45,6 @@ class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: Gener } ) - import scala.collection.JavaConverters._ val partitionInfo = consumer.partitionsFor(topic).asScala val topicPartitions = partitionInfo.map(p => new TopicPartition(p.topic(), p.partition())) consumer From bc59081a94a3cc408f05dbb056b30d0004dd0288 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 1 Jul 2022 17:03:59 +0200 Subject: [PATCH 14/34] Use futures, todo: return false if kafka topic doesn't exist --- .../HyperdriveOffsetComparisonService.scala | 109 +++++++++++------- ...yperdriveOffsetComparisonServiceTest.scala | 89 ++++++++------ 2 files changed, 119 insertions(+), 79 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 379efefce..c0d4653b1 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -28,13 +28,13 @@ import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanc import java.util.Properties import javax.inject.Inject -import scala.util.Try +import scala.concurrent.{ExecutionContext, Future} trait HyperdriveOffsetComparisonService { def getResolvedAppArguments(jobDefinition: ResolvedJobDefinition): Option[Map[String, String]] def getHdfsParameters(resolvedAppArguments: Map[String, String]): Option[HdfsParameters] def getKafkaParameters(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] - def isNewJobInstanceRequired(jobDefinition: ResolvedJobDefinition): Boolean + def isNewJobInstanceRequired(jobDefinition: ResolvedJobDefinition)(implicit ec: ExecutionContext): Future[Boolean] } @Service @@ -51,7 +51,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private val ListDelimiter = ',' override def getResolvedAppArguments(jobDefinition: ResolvedJobDefinition): Option[Map[String, String]] = { - if (isHyperdriveJob(jobDefinition)) { + if (!isHyperdriveJob(jobDefinition)) { logger.warn(s"Job Definition ${jobDefinition} is not a Hyperdrive Job!") None } else { @@ -84,7 +84,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } override def getKafkaParameters(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] = { - if (isHyperdriveJob(jobDefinition)) { + if (!isHyperdriveJob(jobDefinition)) { logger.warn(s"Job Definition ${jobDefinition} is not a Hyperdrive Job!") None } else { @@ -124,52 +124,79 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } private def isHyperdriveJob(jobDefinition: ResolvedJobDefinition) = - jobDefinition.jobParameters.jobType != JobTypes.Hyperdrive || - !jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] - - override def isNewJobInstanceRequired(jobDefinition: ResolvedJobDefinition): Boolean = { - val latestOffsetFilePath = for { - resolvedAppArguments <- getResolvedAppArguments(jobDefinition) - hdfsParameters <- getHdfsParameters(resolvedAppArguments) - latestOffsetFilePath <- hdfsService.getLatestOffsetFilePath(hdfsParameters) - } yield latestOffsetFilePath - - if (latestOffsetFilePath.isEmpty || !latestOffsetFilePath.get._2) { - logger.debug( - s"New job instance required because offset does not exist or is not committed ${latestOffsetFilePath}" - ) - true - } else { - val allKafkaOffsetsConsumedOpt = for { - kafkaParameters <- getKafkaParameters(jobDefinition) - hdfsAllOffsets <- Try( - hdfsService.parseFileAndClose(latestOffsetFilePath.get._1, hdfsService.parseKafkaOffsetStream) - ) - .recover { case e: Exception => - logger.warn(s"Couldn't parse file ${latestOffsetFilePath.get._1}", e) - None - } - .toOption - .flatten - hdfsOffsets <- hdfsAllOffsets.get(kafkaParameters._1) match { + jobDefinition.jobParameters.jobType == JobTypes.Hyperdrive && + jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] + + def isNewJobInstanceRequired(jobDefinition: ResolvedJobDefinition)(implicit ec: ExecutionContext): Future[Boolean] = { + val hdfsParametersOpt = getResolvedAppArguments(jobDefinition).flatMap(getHdfsParameters) + val kafkaParametersOpt = getKafkaParameters(jobDefinition) + + if (hdfsParametersOpt.isEmpty) { + logger.debug(s"Hdfs parameters were not found in job definition ${jobDefinition}") + } + if (kafkaParametersOpt.isEmpty) { + logger.debug(s"Kafka parameters were not found in job definition ${jobDefinition}") + } + + val hdfsOffsetsOptFut: Future[Option[Map[Int, Long]]] = Future { + val latestOffsetOpt = for { + hdfsParameters <- hdfsParametersOpt + latestOffset <- hdfsService.getLatestOffsetFilePath(hdfsParameters) + } yield { latestOffset } + if (latestOffsetOpt.isEmpty || !latestOffsetOpt.get._2) { + logger.debug(s"Offset does not exist or is not committed ${latestOffsetOpt}") + None + } else { + latestOffsetOpt + } + }.flatMap { + case None => Future { None } + case Some(_) if kafkaParametersOpt.isEmpty => Future { None } + case Some(latestOffset) => Future { + hdfsService.parseFileAndClose(latestOffset._1, hdfsService.parseKafkaOffsetStream) + }.recover { case e: Exception => + logger.warn(s"Couldn't parse file ${latestOffset._1}", e) + None + } + }.map { hdfsAllOffsetsOpt => + hdfsAllOffsetsOpt.flatMap { hdfsAllOffsets => + val kafkaParameters = kafkaParametersOpt.get + hdfsAllOffsets.get(kafkaParameters._1) match { case Some(v) => Some(v) case None => logger.warn(s"Could not find offsets for topic ${kafkaParameters._1} in hdfs offsets ${hdfsAllOffsets}") None } - kafkaOffsets = kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) - } yield { - val isSamePartitions = kafkaOffsets.keySet == hdfsOffsets.keySet - isSamePartitions && kafkaOffsets.forall { case (partition, kafkaPartitionOffset) => - hdfsOffsets(partition) == kafkaPartitionOffset - } } + } + + val kafkaOffsetsOptFut: Future[Option[Map[Int, Long]]] = Future { + kafkaParametersOpt.map { kafkaParameters => + kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) + } + } - allKafkaOffsetsConsumedOpt match { - case Some(allKafkaOffsetsConsumed) => !allKafkaOffsetsConsumed - case None => true + val isNewJobInstanceRequiredFut = for { + hdfsOffsetsOpt <- hdfsOffsetsOptFut + kafkaOffsetsOpt <- kafkaOffsetsOptFut + } yield { + (hdfsOffsetsOpt, kafkaOffsetsOpt) match { + case (Some(hdfsOffsets), Some(kafkaOffsets)) => + val isSamePartitions = kafkaOffsets.keySet == hdfsOffsets.keySet + val allOffsetsConsumed = isSamePartitions && kafkaOffsets.nonEmpty && kafkaOffsets.forall { case (partition, kafkaPartitionOffset) => + hdfsOffsets(partition) == kafkaPartitionOffset + } + !allOffsetsConsumed + // TODO: Add logic to return false when topic doesn't exist, or topic is empty + case _ => true } } + + isNewJobInstanceRequiredFut.recover { + case e: Exception => + logger.warn("An error occurred while getting offsets", e) + true + } } private def parseConfiguration(settings: Array[String]): Configuration = { diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala index c94fee5a8..e36ff04fc 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -17,14 +17,14 @@ package za.co.absa.hyperdrive.trigger.api.rest.services import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, eq => eqTo} -import org.mockito.Mockito.{never, reset, verify, when} +import org.mockito.Mockito.{reset, verify, when} import org.scalatest.mockito.MockitoSugar -import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} +import org.scalatest.{AsyncFlatSpec, BeforeAndAfter, Matchers} import za.co.absa.hyperdrive.trigger.configuration.application.DefaultTestSparkConfig import za.co.absa.hyperdrive.trigger.models.enums.JobTypes import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, ShellInstanceParameters, SparkInstanceParameters} -class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { +class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers with BeforeAndAfter with MockitoSugar { private val hdfsService = mock[HdfsService] private val kafkaService = mock[KafkaService] private val underTest = @@ -217,16 +217,18 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with .thenReturn(Option(Map("some-topic" -> Map(2 -> 2021L, 0 -> 21L, 1 -> 1021L)))) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) - val result = underTest.isNewJobInstanceRequired(jobDefinition) - - val hdfsParametersCaptor: ArgumentCaptor[HdfsParameters] = ArgumentCaptor.forClass(classOf[HdfsParameters]) - verify(hdfsService).getLatestOffsetFilePath(hdfsParametersCaptor.capture()) - hdfsParametersCaptor.getValue.keytab shouldBe "/path/to/keytab" - hdfsParametersCaptor.getValue.principal shouldBe "principal" - hdfsParametersCaptor.getValue.checkpointLocation shouldBe "/checkpoint/path/some-topic" - verify(hdfsService).parseFileAndClose(eqTo("/checkpoint/path/some-topic/offsets/21"), any()) - verify(kafkaService).getEndOffsets(eqTo("some-topic"), any()) - result shouldBe false + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + + resultFut.map { result => + val hdfsParametersCaptor: ArgumentCaptor[HdfsParameters] = ArgumentCaptor.forClass(classOf[HdfsParameters]) + verify(hdfsService).getLatestOffsetFilePath(hdfsParametersCaptor.capture()) + hdfsParametersCaptor.getValue.keytab shouldBe "/path/to/keytab" + hdfsParametersCaptor.getValue.principal shouldBe "principal" + hdfsParametersCaptor.getValue.checkpointLocation shouldBe "/checkpoint/path/some-topic" + verify(hdfsService).parseFileAndClose(eqTo("/checkpoint/path/some-topic/offsets/21"), any()) + verify(kafkaService).getEndOffsets(eqTo("some-topic"), any()) + result shouldBe false + } } it should "return true if no offset file is present" in { @@ -251,10 +253,12 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with ) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(None) - val result = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) - verify(hdfsService).getLatestOffsetFilePath(any()) - result shouldBe true + resultFut.map { result => + verify(hdfsService).getLatestOffsetFilePath(any()) + result shouldBe true + } } it should "return true if the offset is not committed" in { @@ -279,10 +283,12 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with ) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", false))) - val result = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) - verify(hdfsService).getLatestOffsetFilePath(any()) - result shouldBe true + resultFut.map { result => + verify(hdfsService).getLatestOffsetFilePath(any()) + result shouldBe true + } } it should "return true if a offset file could not be parsed" in { @@ -312,11 +318,13 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) .thenThrow(new RuntimeException("Failed to parse")) - val result = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) - verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).parseFileAndClose(any(), any()) - result shouldBe true + resultFut.map { result => + verify(hdfsService).getLatestOffsetFilePath(any()) + verify(hdfsService).parseFileAndClose(any(), any()) + result shouldBe true + } } it should "return true if the checkpoints offset does not contain the topic" in { @@ -345,12 +353,13 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) .thenReturn(Option(Map("some-other-topic" -> Map(0 -> 21L)))) - val result = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) - verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).parseFileAndClose(any(), any()) - verify(kafkaService, never()).getEndOffsets(any(), any()) - result shouldBe true + resultFut.map { result => + verify(hdfsService).getLatestOffsetFilePath(any()) + verify(hdfsService).parseFileAndClose(any(), any()) + result shouldBe true + } } it should "return true if the kafka offsets and checkpoint offset do not have the same set of partitions" in { @@ -381,12 +390,14 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with .thenReturn(Option(Map("some-topic" -> Map(0 -> 21L)))) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) - val result = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) - verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).parseFileAndClose(any(), any()) - verify(kafkaService).getEndOffsets(any(), any()) - result shouldBe true + resultFut.map { result => + verify(hdfsService).getLatestOffsetFilePath(any()) + verify(hdfsService).parseFileAndClose(any(), any()) + verify(kafkaService).getEndOffsets(any(), any()) + result shouldBe true + } } it should "return true if the kafka offsets and checkpoint offsets are not the same" in { @@ -417,11 +428,13 @@ class HyperdriveOffsetComparisonServiceTest extends FlatSpec with Matchers with .thenReturn(Option(Map("some-topic" -> Map(0 -> 42L, 1 -> 55L)))) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 42L, 1 -> 7L)) - val result = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) - verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).parseFileAndClose(any(), any()) - verify(kafkaService).getEndOffsets(any(), any()) - result shouldBe true + resultFut.map { result => + verify(hdfsService).getLatestOffsetFilePath(any()) + verify(hdfsService).parseFileAndClose(any(), any()) + verify(kafkaService).getEndOffsets(any(), any()) + result shouldBe true + } } } From a7cf78ffdb672a598a8563785a1aa65e61a0eda1 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 13 Jul 2022 13:52:13 +0200 Subject: [PATCH 15/34] return false if kafka topic doesn't exist. Add tests --- .../HyperdriveOffsetComparisonService.scala | 109 +++++---- .../api/rest/services/KafkaService.scala | 48 ++-- .../rest/services/HyperdriveKafkaTest.scala | 43 ++++ ...yperdriveOffsetComparisonServiceTest.scala | 217 +++++++----------- .../api/rest/services/KafkaServiceTest.scala | 76 ++++++ 5 files changed, 304 insertions(+), 189 deletions(-) create mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveKafkaTest.scala create mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index c0d4653b1..66af92b91 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -128,17 +128,58 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] def isNewJobInstanceRequired(jobDefinition: ResolvedJobDefinition)(implicit ec: ExecutionContext): Future[Boolean] = { - val hdfsParametersOpt = getResolvedAppArguments(jobDefinition).flatMap(getHdfsParameters) val kafkaParametersOpt = getKafkaParameters(jobDefinition) + if (kafkaParametersOpt.isEmpty) { + logger.debug(s"Kafka parameters were not found in job definition ${jobDefinition}") + } + + val kafkaEndOffsetsOptFut = Future { + kafkaParametersOpt.map { kafkaParameters => + kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) + } + } + + val kafkaBeginningOffsetsOptFut = Future { + kafkaParametersOpt.map { kafkaParameters => + kafkaService.getBeginningOffsets(kafkaParameters._1, kafkaParameters._2) + } + } + + val isNewJobInstanceRequiredFut = kafkaEndOffsetsOptFut.flatMap { kafkaEndOffsetsOpt => + kafkaBeginningOffsetsOptFut.flatMap { kafkaBeginningOffsetsOpt => + (kafkaBeginningOffsetsOpt, kafkaEndOffsetsOpt) match { + case (Some(kafkaBeginningOffsets), Some(kafkaEndOffsets)) => + if (kafkaBeginningOffsets.isEmpty) { // topic does not exist + Future { false } + } else if (offsetsEqual(kafkaBeginningOffsets, kafkaEndOffsets)) { // topic is empty + Future { false } + } else { + getCheckpointOffsets(jobDefinition, kafkaParametersOpt).map { + case Some(checkpointOffsets) => !offsetsConsumed(checkpointOffsets, kafkaEndOffsets) + case _ => true + } + } + case _ => Future { true } + } + } + } + + isNewJobInstanceRequiredFut.recover { + case e: Exception => + logger.warn("An error occurred while getting offsets", e) + true + } + } + + private def getCheckpointOffsets(jobDefinition: ResolvedJobDefinition, kafkaParametersOpt: Option[(String, Properties)]) + (implicit ec: ExecutionContext): Future[Option[Map[Int, Long]]] = { + val hdfsParametersOpt = getResolvedAppArguments(jobDefinition).flatMap(getHdfsParameters) if (hdfsParametersOpt.isEmpty) { logger.debug(s"Hdfs parameters were not found in job definition ${jobDefinition}") } - if (kafkaParametersOpt.isEmpty) { - logger.debug(s"Kafka parameters were not found in job definition ${jobDefinition}") - } - val hdfsOffsetsOptFut: Future[Option[Map[Int, Long]]] = Future { + Future { val latestOffsetOpt = for { hdfsParameters <- hdfsParametersOpt latestOffset <- hdfsService.getLatestOffsetFilePath(hdfsParameters) @@ -150,14 +191,14 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, latestOffsetOpt } }.flatMap { - case None => Future { None } - case Some(_) if kafkaParametersOpt.isEmpty => Future { None } - case Some(latestOffset) => Future { - hdfsService.parseFileAndClose(latestOffset._1, hdfsService.parseKafkaOffsetStream) - }.recover { case e: Exception => - logger.warn(s"Couldn't parse file ${latestOffset._1}", e) - None - } + case None => Future { None } + case Some(_) if kafkaParametersOpt.isEmpty => Future { None } + case Some(latestOffset) => Future { + hdfsService.parseFileAndClose(latestOffset._1, hdfsService.parseKafkaOffsetStream) + }.recover { case e: Exception => + logger.warn(s"Couldn't parse file ${latestOffset._1}", e) + None + } }.map { hdfsAllOffsetsOpt => hdfsAllOffsetsOpt.flatMap { hdfsAllOffsets => val kafkaParameters = kafkaParametersOpt.get @@ -169,34 +210,6 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } } } - - val kafkaOffsetsOptFut: Future[Option[Map[Int, Long]]] = Future { - kafkaParametersOpt.map { kafkaParameters => - kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) - } - } - - val isNewJobInstanceRequiredFut = for { - hdfsOffsetsOpt <- hdfsOffsetsOptFut - kafkaOffsetsOpt <- kafkaOffsetsOptFut - } yield { - (hdfsOffsetsOpt, kafkaOffsetsOpt) match { - case (Some(hdfsOffsets), Some(kafkaOffsets)) => - val isSamePartitions = kafkaOffsets.keySet == hdfsOffsets.keySet - val allOffsetsConsumed = isSamePartitions && kafkaOffsets.nonEmpty && kafkaOffsets.forall { case (partition, kafkaPartitionOffset) => - hdfsOffsets(partition) == kafkaPartitionOffset - } - !allOffsetsConsumed - // TODO: Add logic to return false when topic doesn't exist, or topic is empty - case _ => true - } - } - - isNewJobInstanceRequiredFut.recover { - case e: Exception => - logger.warn("An error occurred while getting offsets", e) - true - } } private def parseConfiguration(settings: Array[String]): Configuration = { @@ -220,4 +233,18 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, configuration.setProperty(settingKeyValue(0).trim, settingKeyValue(1).trim) } } + + private def offsetsEqual(offsets1: Map[Int, Long], offsets2: Map[Int, Long]) = { + offsets1.keySet == offsets2.keySet && + offsets1.forall { case (partition, offset1) => + offset1 == offsets2(partition) + } + } + + private def offsetsConsumed(checkpointOffsets: Map[Int, Long], kafkaOffsets: Map[Int, Long]) = { + val isSamePartitions = kafkaOffsets.keySet == checkpointOffsets.keySet + isSamePartitions && kafkaOffsets.nonEmpty && kafkaOffsets.forall { case (partition, kafkaPartitionOffset) => + checkpointOffsets(partition) == kafkaPartitionOffset + } + } } diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index 79636c1f5..b8262e1ca 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -18,41 +18,61 @@ package za.co.absa.hyperdrive.trigger.api.rest.services import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.common.TopicPartition import org.springframework.stereotype.Service +import za.co.absa.hyperdrive.trigger.api.rest.services.KafkaServiceImpl.{BeginningOffsets, EndOffsets, OffsetFunction} import za.co.absa.hyperdrive.trigger.configuration.application.{GeneralConfig, KafkaConfig} import java.util.Properties import java.util.concurrent.ConcurrentHashMap import javax.inject.Inject +import scala.collection.JavaConverters._ trait KafkaService { + def getBeginningOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] } @Service class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) extends KafkaService { + private val kafkaConsumersCache = new ConcurrentHashMap[Properties, KafkaConsumer[String, String]]() + override def getBeginningOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] = { + getOffsets(topic, consumerProperties, BeginningOffsets) + } + override def getEndOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] = { - val groupId = s"${kafkaConfig.groupIdPrefix}-${generalConfig.appUniqueId}-getEndOffsets" - consumerProperties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) - import scala.collection.JavaConverters._ + getOffsets(topic, consumerProperties, EndOffsets) + } + + def createKafkaConsumer(properties: Properties) = new KafkaConsumer[String, String](properties) + + private def getOffsets(topic: String, properties: Properties, offsetFn: OffsetFunction): Map[Int, Long] = { + val groupId = s"${kafkaConfig.groupIdPrefix}-${generalConfig.appUniqueId}-kafkaService" + properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) val consumer = kafkaConsumersCache .asScala - .getOrElse(consumerProperties, { - val consumer = new KafkaConsumer[String, String](consumerProperties) - kafkaConsumersCache.put(consumerProperties, consumer) - consumer - } - ) - - val partitionInfo = consumer.partitionsFor(topic).asScala + .getOrElse(properties, { + val consumer = createKafkaConsumer(properties) + kafkaConsumersCache.put(properties, consumer) + consumer + }) + val partitionInfo = Option(consumer.partitionsFor(topic)).map(_.asScala).getOrElse(Seq()) val topicPartitions = partitionInfo.map(p => new TopicPartition(p.topic(), p.partition())) - consumer - .endOffsets(topicPartitions.asJava) - .asScala + val offsets = offsetFn match { + case KafkaServiceImpl.BeginningOffsets => consumer.beginningOffsets(topicPartitions.asJava) + case KafkaServiceImpl.EndOffsets => consumer.endOffsets(topicPartitions.asJava) + } + Option(offsets).map(_.asScala).getOrElse(Map()) .map { case (topicPartition: TopicPartition, offset: java.lang.Long) => topicPartition.partition() -> offset.longValue() } .toMap } + +} + +object KafkaServiceImpl { + sealed abstract class OffsetFunction + case object BeginningOffsets extends OffsetFunction + case object EndOffsets extends OffsetFunction } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveKafkaTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveKafkaTest.scala new file mode 100644 index 000000000..54a4b9321 --- /dev/null +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveKafkaTest.scala @@ -0,0 +1,43 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.scalatest.mockito.MockitoSugar +import org.scalatest.{AsyncFlatSpec, BeforeAndAfter, Matchers} +import za.co.absa.hyperdrive.trigger.configuration.application.{KafkaConfig, TestGeneralConfig} + +import java.util.Properties + +class HyperdriveKafkaTest extends AsyncFlatSpec with Matchers with BeforeAndAfter with MockitoSugar { + private val kafkaProperties = { + val properties = new Properties() + properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") + properties.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer") + properties.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer") + properties.put("max.poll.records", "100") + properties + } + + private val kafkaConfig = new KafkaConfig(new Properties(), "group.id.prefix", 500L) + private val generalConfig = TestGeneralConfig() + private val kafkaService = new KafkaServiceImpl(kafkaConfig, generalConfig) + + "getResolvedAppArguments" should "return the resolved app arguments" in { + val result = kafkaService.getEndOffsets("test-topic", kafkaProperties) + succeed + } +} diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala index e36ff04fc..dee4423d5 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -17,7 +17,7 @@ package za.co.absa.hyperdrive.trigger.api.rest.services import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, eq => eqTo} -import org.mockito.Mockito.{reset, verify, when} +import org.mockito.Mockito.{never, reset, verify, when} import org.scalatest.mockito.MockitoSugar import org.scalatest.{AsyncFlatSpec, BeforeAndAfter, Matchers} import za.co.absa.hyperdrive.trigger.configuration.application.DefaultTestSparkConfig @@ -190,31 +190,14 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } "isNewJobInstanceRequired" should "return false if the kafka and checkpoint folder offsets are the same" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = - Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - ) - ) + val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=some-topic", - "reader.kafka.brokers=http://localhost:9092", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 - ) + val jobDefinition = getJobDefinition when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("/checkpoint/path/some-topic/offsets/21", true))) when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) .thenReturn(Option(Map("some-topic" -> Map(2 -> 2021L, 0 -> 21L, 1 -> 1021L)))) + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) @@ -231,26 +214,45 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } } + it should "return false if the kafka topic does not exist" in { + val config = getSparkConfig + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = getJobDefinition + + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map[Int, Long]()) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map[Int, Long]()) + + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + + resultFut.map { result => + verify(hdfsService, never()).getLatestOffsetFilePath(any()) + result shouldBe false + } + } + + it should "return false if the kafka topic is empty" in { + val config = getSparkConfig + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = getJobDefinition + + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 42L)) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 42L)) + + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + + resultFut.map { result => + verify(hdfsService, never()).getLatestOffsetFilePath(any()) + result shouldBe false + } + } + it should "return true if no offset file is present" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = - Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - ) - ) + val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 - ) + val jobDefinition = getJobDefinition + + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(None) val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) @@ -262,25 +264,12 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } it should "return true if the offset is not committed" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = - Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - ) - ) + val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 - ) + val jobDefinition = getJobDefinition + + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", false))) val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) @@ -292,28 +281,12 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } it should "return true if a offset file could not be parsed" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = - Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - ) - ) + val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=some-topic", - "reader.kafka.brokers=http://localhost:9092", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 - ) + val jobDefinition = getJobDefinition + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) .thenThrow(new RuntimeException("Failed to parse")) @@ -328,27 +301,12 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } it should "return true if the checkpoints offset does not contain the topic" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = - Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - ) - ) + val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=some-topic", - "reader.kafka.brokers=http://localhost:9092", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 - ) + val jobDefinition = getJobDefinition + + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) .thenReturn(Option(Map("some-other-topic" -> Map(0 -> 21L)))) @@ -363,31 +321,14 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } it should "return true if the kafka offsets and checkpoint offset do not have the same set of partitions" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = - Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - ) - ) + val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=some-topic", - "reader.kafka.brokers=http://localhost:9092", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 - ) + val jobDefinition = getJobDefinition when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) .thenReturn(Option(Map("some-topic" -> Map(0 -> 21L)))) + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) @@ -401,14 +342,36 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } it should "return true if the kafka offsets and checkpoint offsets are not the same" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = + val config = getSparkConfig + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val jobDefinition = getJobDefinition + + when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) + when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + .thenReturn(Option(Map("some-topic" -> Map(0 -> 42L, 1 -> 55L)))) + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 0L)) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 42L, 1 -> 7L)) + + val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + + resultFut.map { result => + verify(hdfsService).getLatestOffsetFilePath(any()) + verify(hdfsService).parseFileAndClose(any(), any()) + verify(kafkaService).getEndOffsets(any(), any()) + result shouldBe true + } + } + + private def getSparkConfig = + DefaultTestSparkConfig().copy(additionalConfs = Map( "spark.yarn.keytab" -> "/path/to/keytab", "spark.yarn.principal" -> "principal" ) ) - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = ResolvedJobDefinition( + + private def getJobDefinition = { + ResolvedJobDefinition( "job", SparkInstanceParameters( jobType = JobTypes.Hyperdrive, @@ -422,19 +385,5 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers ), order = 1 ) - - when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) - .thenReturn(Option(Map("some-topic" -> Map(0 -> 42L, 1 -> 55L)))) - when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 42L, 1 -> 7L)) - - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) - - resultFut.map { result => - verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).parseFileAndClose(any(), any()) - verify(kafkaService).getEndOffsets(any(), any()) - result shouldBe true - } } } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala new file mode 100644 index 000000000..f579d5da1 --- /dev/null +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala @@ -0,0 +1,76 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.apache.kafka.clients.consumer.KafkaConsumer +import org.apache.kafka.common.{PartitionInfo, TopicPartition} +import org.mockito.ArgumentMatchers.{any, eq => eqTo} +import org.mockito.Mockito.when +import org.scalatest.mockito.MockitoSugar +import org.scalatest.{FlatSpec, Matchers} +import za.co.absa.hyperdrive.trigger.configuration.application.{GeneralConfig, KafkaConfig, TestGeneralConfig, TestKafkaConfig} + +import java.util.Properties + +class KafkaServiceTest extends FlatSpec with MockitoSugar with Matchers { + + private val mockKafkaConsumer = mock[KafkaConsumer[String, String]] + class KafkaServiceTestImpl(kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) extends KafkaServiceImpl(kafkaConfig, generalConfig) { + override def createKafkaConsumer(properties: Properties): KafkaConsumer[String, String] = mockKafkaConsumer + } + private val underTest = new KafkaServiceTestImpl(TestKafkaConfig(), TestGeneralConfig()) + + "getEndOffsets" should "return a map of end offsets" in { + import scala.collection.JavaConverters._ + val partitions = Seq( + new PartitionInfo("topic", 0, null, null, null), + new PartitionInfo("topic", 1, null, null, null) + ) + val endOffsets = Map( + new TopicPartition("topic", 0) -> long2Long(200L), + new TopicPartition("topic", 1) -> long2Long(400L) + ).asJava + val topicPartitions = partitions.map(p => new TopicPartition(p.topic(), p.partition())).asJava + + when(mockKafkaConsumer.partitionsFor(any())).thenReturn(partitions.asJava) + when(mockKafkaConsumer.endOffsets(eqTo(topicPartitions))).thenReturn(endOffsets) + + val result = underTest.getEndOffsets("topic", new Properties()) + + result shouldBe Map(0 -> 200L, 1 -> 400L) + } + + it should "return an empty map if partitionsFor returns null" in { + when(mockKafkaConsumer.partitionsFor(any())).thenReturn(null) + + val result = underTest.getEndOffsets("non-existent-topic", new Properties()) + + result shouldBe Map() + } + + it should "return an empty map if getOffsets returns null" in { + val partitionInfo = new PartitionInfo("topic", 0, null, null, null) + import scala.collection.JavaConverters._ + val partitions = Seq(partitionInfo).asJava + when(mockKafkaConsumer.partitionsFor(any())).thenReturn(partitions) + when(mockKafkaConsumer.endOffsets(any())).thenReturn(null) + + val result = underTest.getEndOffsets("non-existent-topic", new Properties()) + + result shouldBe Map() + } +} From 7955475f00deeddb6d2229c855f277ae573c399b Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 13 Jul 2022 13:53:20 +0200 Subject: [PATCH 16/34] scalafmt --- .../HyperdriveOffsetComparisonService.scala | 35 ++++++++++--------- .../api/rest/services/KafkaService.scala | 18 +++++----- .../api/rest/services/KafkaServiceTest.scala | 4 +-- 3 files changed, 30 insertions(+), 27 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 66af92b91..cf79f4eec 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -156,7 +156,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } else { getCheckpointOffsets(jobDefinition, kafkaParametersOpt).map { case Some(checkpointOffsets) => !offsetsConsumed(checkpointOffsets, kafkaEndOffsets) - case _ => true + case _ => true } } case _ => Future { true } @@ -164,15 +164,15 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } } - isNewJobInstanceRequiredFut.recover { - case e: Exception => - logger.warn("An error occurred while getting offsets", e) - true + isNewJobInstanceRequiredFut.recover { case e: Exception => + logger.warn("An error occurred while getting offsets", e) + true } } - private def getCheckpointOffsets(jobDefinition: ResolvedJobDefinition, kafkaParametersOpt: Option[(String, Properties)]) - (implicit ec: ExecutionContext): Future[Option[Map[Int, Long]]] = { + private def getCheckpointOffsets(jobDefinition: ResolvedJobDefinition, + kafkaParametersOpt: Option[(String, Properties)] + )(implicit ec: ExecutionContext): Future[Option[Map[Int, Long]]] = { val hdfsParametersOpt = getResolvedAppArguments(jobDefinition).flatMap(getHdfsParameters) if (hdfsParametersOpt.isEmpty) { @@ -191,14 +191,15 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, latestOffsetOpt } }.flatMap { - case None => Future { None } + case None => Future { None } case Some(_) if kafkaParametersOpt.isEmpty => Future { None } - case Some(latestOffset) => Future { - hdfsService.parseFileAndClose(latestOffset._1, hdfsService.parseKafkaOffsetStream) - }.recover { case e: Exception => - logger.warn(s"Couldn't parse file ${latestOffset._1}", e) - None - } + case Some(latestOffset) => + Future { + hdfsService.parseFileAndClose(latestOffset._1, hdfsService.parseKafkaOffsetStream) + }.recover { case e: Exception => + logger.warn(s"Couldn't parse file ${latestOffset._1}", e) + None + } }.map { hdfsAllOffsetsOpt => hdfsAllOffsetsOpt.flatMap { hdfsAllOffsets => val kafkaParameters = kafkaParametersOpt.get @@ -236,9 +237,9 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private def offsetsEqual(offsets1: Map[Int, Long], offsets2: Map[Int, Long]) = { offsets1.keySet == offsets2.keySet && - offsets1.forall { case (partition, offset1) => - offset1 == offsets2(partition) - } + offsets1.forall { case (partition, offset1) => + offset1 == offsets2(partition) + } } private def offsetsConsumed(checkpointOffsets: Map[Int, Long], kafkaOffsets: Map[Int, Long]) = { diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index b8262e1ca..685fb9a2f 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -49,20 +49,22 @@ class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: Gener private def getOffsets(topic: String, properties: Properties, offsetFn: OffsetFunction): Map[Int, Long] = { val groupId = s"${kafkaConfig.groupIdPrefix}-${generalConfig.appUniqueId}-kafkaService" properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) - val consumer = kafkaConsumersCache - .asScala + val consumer = kafkaConsumersCache.asScala .getOrElse(properties, { - val consumer = createKafkaConsumer(properties) - kafkaConsumersCache.put(properties, consumer) - consumer - }) + val consumer = createKafkaConsumer(properties) + kafkaConsumersCache.put(properties, consumer) + consumer + } + ) val partitionInfo = Option(consumer.partitionsFor(topic)).map(_.asScala).getOrElse(Seq()) val topicPartitions = partitionInfo.map(p => new TopicPartition(p.topic(), p.partition())) val offsets = offsetFn match { case KafkaServiceImpl.BeginningOffsets => consumer.beginningOffsets(topicPartitions.asJava) - case KafkaServiceImpl.EndOffsets => consumer.endOffsets(topicPartitions.asJava) + case KafkaServiceImpl.EndOffsets => consumer.endOffsets(topicPartitions.asJava) } - Option(offsets).map(_.asScala).getOrElse(Map()) + Option(offsets) + .map(_.asScala) + .getOrElse(Map()) .map { case (topicPartition: TopicPartition, offset: java.lang.Long) => topicPartition.partition() -> offset.longValue() } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala index f579d5da1..6c61c1b8b 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala @@ -1,4 +1,3 @@ - /* * Copyright 2018 ABSA Group Limited * @@ -29,7 +28,8 @@ import java.util.Properties class KafkaServiceTest extends FlatSpec with MockitoSugar with Matchers { private val mockKafkaConsumer = mock[KafkaConsumer[String, String]] - class KafkaServiceTestImpl(kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) extends KafkaServiceImpl(kafkaConfig, generalConfig) { + class KafkaServiceTestImpl(kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) + extends KafkaServiceImpl(kafkaConfig, generalConfig) { override def createKafkaConsumer(properties: Properties): KafkaConsumer[String, String] = mockKafkaConsumer } private val underTest = new KafkaServiceTestImpl(TestKafkaConfig(), TestGeneralConfig()) From eb57234181537d5503a714baf4be054aa4085abb Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 13 Jul 2022 15:01:31 +0200 Subject: [PATCH 17/34] scalafmt --- .../trigger/api/rest/services/KafkaServiceTest.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala index 6c61c1b8b..126e6db10 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala @@ -21,7 +21,12 @@ import org.mockito.ArgumentMatchers.{any, eq => eqTo} import org.mockito.Mockito.when import org.scalatest.mockito.MockitoSugar import org.scalatest.{FlatSpec, Matchers} -import za.co.absa.hyperdrive.trigger.configuration.application.{GeneralConfig, KafkaConfig, TestGeneralConfig, TestKafkaConfig} +import za.co.absa.hyperdrive.trigger.configuration.application.{ + GeneralConfig, + KafkaConfig, + TestGeneralConfig, + TestKafkaConfig +} import java.util.Properties From 15299f0764c977b4fe6b5b2a1cc8059af7582bb9 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 13 Jul 2022 21:26:14 +0200 Subject: [PATCH 18/34] Remove temp file --- .../rest/services/HyperdriveKafkaTest.scala | 43 ------------------- 1 file changed, 43 deletions(-) delete mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveKafkaTest.scala diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveKafkaTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveKafkaTest.scala deleted file mode 100644 index 54a4b9321..000000000 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveKafkaTest.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Copyright 2018 ABSA Group Limited - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package za.co.absa.hyperdrive.trigger.api.rest.services - -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.scalatest.mockito.MockitoSugar -import org.scalatest.{AsyncFlatSpec, BeforeAndAfter, Matchers} -import za.co.absa.hyperdrive.trigger.configuration.application.{KafkaConfig, TestGeneralConfig} - -import java.util.Properties - -class HyperdriveKafkaTest extends AsyncFlatSpec with Matchers with BeforeAndAfter with MockitoSugar { - private val kafkaProperties = { - val properties = new Properties() - properties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") - properties.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer") - properties.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer") - properties.put("max.poll.records", "100") - properties - } - - private val kafkaConfig = new KafkaConfig(new Properties(), "group.id.prefix", 500L) - private val generalConfig = TestGeneralConfig() - private val kafkaService = new KafkaServiceImpl(kafkaConfig, generalConfig) - - "getResolvedAppArguments" should "return the resolved app arguments" in { - val result = kafkaService.getEndOffsets("test-topic", kafkaProperties) - succeed - } -} From cddc675c0d71948128327a6b99ea80a5dc3e5b58 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 13 Jul 2022 21:35:44 +0200 Subject: [PATCH 19/34] login explicitly --- .../trigger/api/rest/services/HdfsService.scala | 10 +++++++--- .../services/HyperdriveOffsetComparisonService.scala | 1 + 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala index df7e8fa67..336799349 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -33,6 +33,7 @@ trait HdfsService { def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] + def loginUserFromKeytab(principal: String, keytab: String): Unit } class HdfsParameters( @@ -124,7 +125,6 @@ class HdfsServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformati * commit file is assumed to also not exist. */ override def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] = { - userGroupInformationWrapper.loginUserFromKeytab(params.principal, params.keytab) val offsetBatchIdOpt = getLatestOffsetBatchId(params.checkpointLocation) val offsetFilePath = offsetBatchIdOpt.map { offsetBatchId => val commitBatchIdOpt = getLatestCommitBatchId(params.checkpointLocation) @@ -141,12 +141,16 @@ class HdfsServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformati offsetFilePath } - def getLatestCommitBatchId(checkpointDir: String): Option[Long] = { + override def loginUserFromKeytab(principal: String, keytab: String): Unit = { + userGroupInformationWrapper.loginUserFromKeytab(principal, keytab) + } + + private def getLatestCommitBatchId(checkpointDir: String): Option[Long] = { val commitsDir = new Path(s"$checkpointDir/$commitsDirName") getLatestBatchId(commitsDir) } - def getLatestOffsetBatchId(checkpointDir: String): Option[Long] = { + private def getLatestOffsetBatchId(checkpointDir: String): Option[Long] = { val offsetsDir = new Path(s"$checkpointDir/$offsetsDirName") getLatestBatchId(offsetsDir) } diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index cf79f4eec..add485d90 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -182,6 +182,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, Future { val latestOffsetOpt = for { hdfsParameters <- hdfsParametersOpt + _ = hdfsService.loginUserFromKeytab(hdfsParameters.principal, hdfsParameters.keytab) latestOffset <- hdfsService.getLatestOffsetFilePath(hdfsParameters) } yield { latestOffset } if (latestOffsetOpt.isEmpty || !latestOffsetOpt.get._2) { From 541f8d187726a2ce7b3b005547fcb69e6109e489 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 14 Jul 2022 11:59:27 +0200 Subject: [PATCH 20/34] Change parameter type to JobInstanceParameters --- .../HyperdriveOffsetComparisonService.scala | 48 ++--- ...yperdriveOffsetComparisonServiceTest.scala | 190 ++++++++---------- 2 files changed, 103 insertions(+), 135 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index add485d90..946345611 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -24,17 +24,17 @@ import org.slf4j.LoggerFactory import org.springframework.stereotype.Service import za.co.absa.hyperdrive.trigger.configuration.application.SparkConfig import za.co.absa.hyperdrive.trigger.models.enums.JobTypes -import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, SparkInstanceParameters} +import za.co.absa.hyperdrive.trigger.models.{JobInstanceParameters, SparkInstanceParameters} import java.util.Properties import javax.inject.Inject import scala.concurrent.{ExecutionContext, Future} trait HyperdriveOffsetComparisonService { - def getResolvedAppArguments(jobDefinition: ResolvedJobDefinition): Option[Map[String, String]] + def getResolvedAppArguments(jobParameters: JobInstanceParameters): Option[Map[String, String]] def getHdfsParameters(resolvedAppArguments: Map[String, String]): Option[HdfsParameters] - def getKafkaParameters(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] - def isNewJobInstanceRequired(jobDefinition: ResolvedJobDefinition)(implicit ec: ExecutionContext): Future[Boolean] + def getKafkaParameters(jobParameters: JobInstanceParameters): Option[(String, Properties)] + def isNewJobInstanceRequired(jobParameters: JobInstanceParameters)(implicit ec: ExecutionContext): Future[Boolean] } @Service @@ -50,13 +50,13 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private val PropertyDelimiter = "=" private val ListDelimiter = ',' - override def getResolvedAppArguments(jobDefinition: ResolvedJobDefinition): Option[Map[String, String]] = { - if (!isHyperdriveJob(jobDefinition)) { - logger.warn(s"Job Definition ${jobDefinition} is not a Hyperdrive Job!") + override def getResolvedAppArguments(jobParameters: JobInstanceParameters): Option[Map[String, String]] = { + if (!isHyperdriveJob(jobParameters)) { + logger.warn(s"Job Parameters ${jobParameters} is not a Hyperdrive Job!") None } else { - val jobParameters = jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters] - val args = jobParameters.appArguments + val sparkParameters = jobParameters.asInstanceOf[SparkInstanceParameters] + val args = sparkParameters.appArguments val config = parseConfiguration(args.toArray) import scala.collection.JavaConverters._ val resolvedArgs = config.getKeys.asScala.map { k => @@ -83,12 +83,12 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, hdfsParameters } - override def getKafkaParameters(jobDefinition: ResolvedJobDefinition): Option[(String, Properties)] = { - if (!isHyperdriveJob(jobDefinition)) { - logger.warn(s"Job Definition ${jobDefinition} is not a Hyperdrive Job!") + override def getKafkaParameters(jobParameters: JobInstanceParameters): Option[(String, Properties)] = { + if (!isHyperdriveJob(jobParameters)) { + logger.warn(s"Job Definition ${jobParameters} is not a Hyperdrive Job!") None } else { - val args = jobDefinition.jobParameters.asInstanceOf[SparkInstanceParameters].appArguments + val args = jobParameters.asInstanceOf[SparkInstanceParameters].appArguments val kafkaParameters = for { topic <- args .find(_.startsWith(s"$HyperdriveKafkaTopicKey=")) @@ -116,21 +116,21 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, if (kafkaParameters.isEmpty) { logger.warn( - s"Could not find required kafka parameters in job definition ${jobDefinition.name} with args ${args}" + s"Could not find required kafka parameters in job parameters ${jobParameters} with args ${args}" ) } kafkaParameters } } - private def isHyperdriveJob(jobDefinition: ResolvedJobDefinition) = - jobDefinition.jobParameters.jobType == JobTypes.Hyperdrive && - jobDefinition.jobParameters.isInstanceOf[SparkInstanceParameters] + private def isHyperdriveJob(jobParameters: JobInstanceParameters) = + jobParameters.jobType == JobTypes.Hyperdrive && + jobParameters.isInstanceOf[SparkInstanceParameters] - def isNewJobInstanceRequired(jobDefinition: ResolvedJobDefinition)(implicit ec: ExecutionContext): Future[Boolean] = { - val kafkaParametersOpt = getKafkaParameters(jobDefinition) + def isNewJobInstanceRequired(jobParameters: JobInstanceParameters)(implicit ec: ExecutionContext): Future[Boolean] = { + val kafkaParametersOpt = getKafkaParameters(jobParameters) if (kafkaParametersOpt.isEmpty) { - logger.debug(s"Kafka parameters were not found in job definition ${jobDefinition}") + logger.debug(s"Kafka parameters were not found in job definition ${jobParameters}") } val kafkaEndOffsetsOptFut = Future { @@ -154,7 +154,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } else if (offsetsEqual(kafkaBeginningOffsets, kafkaEndOffsets)) { // topic is empty Future { false } } else { - getCheckpointOffsets(jobDefinition, kafkaParametersOpt).map { + getCheckpointOffsets(jobParameters, kafkaParametersOpt).map { case Some(checkpointOffsets) => !offsetsConsumed(checkpointOffsets, kafkaEndOffsets) case _ => true } @@ -170,13 +170,13 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } } - private def getCheckpointOffsets(jobDefinition: ResolvedJobDefinition, + private def getCheckpointOffsets(jobParameters: JobInstanceParameters, kafkaParametersOpt: Option[(String, Properties)] )(implicit ec: ExecutionContext): Future[Option[Map[Int, Long]]] = { - val hdfsParametersOpt = getResolvedAppArguments(jobDefinition).flatMap(getHdfsParameters) + val hdfsParametersOpt = getResolvedAppArguments(jobParameters).flatMap(getHdfsParameters) if (hdfsParametersOpt.isEmpty) { - logger.debug(s"Hdfs parameters were not found in job definition ${jobDefinition}") + logger.debug(s"Hdfs parameters were not found in job definition ${jobParameters}") } Future { diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala index dee4423d5..00cb21ba2 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -22,7 +22,7 @@ import org.scalatest.mockito.MockitoSugar import org.scalatest.{AsyncFlatSpec, BeforeAndAfter, Matchers} import za.co.absa.hyperdrive.trigger.configuration.application.DefaultTestSparkConfig import za.co.absa.hyperdrive.trigger.models.enums.JobTypes -import za.co.absa.hyperdrive.trigger.models.{ResolvedJobDefinition, ShellInstanceParameters, SparkInstanceParameters} +import za.co.absa.hyperdrive.trigger.models.{ShellInstanceParameters, SparkInstanceParameters} class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers with BeforeAndAfter with MockitoSugar { private val hdfsService = mock[HdfsService] @@ -36,20 +36,16 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } "getResolvedAppArguments" should "return the resolved app arguments" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=my-topic", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 + val jobParameters = SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=my-topic", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) ) - val result = underTest.getResolvedAppArguments(jobDefinition) + val result = underTest.getResolvedAppArguments(jobParameters) result.isDefined shouldBe true result.get should contain theSameElementsAs Map( @@ -59,21 +55,17 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } it should "return None if the job type is not hyperdrive" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Spark, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=my-topic", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 + val jobParameters = SparkInstanceParameters( + jobType = JobTypes.Spark, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=my-topic", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) ) - val result = underTest.getResolvedAppArguments(jobDefinition) + val result = underTest.getResolvedAppArguments(jobParameters) result shouldBe None } @@ -99,22 +91,18 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } "getKafkaParameters" should "get kafka properties from a resolved job definition" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=some-topic", - "reader.kafka.brokers=http://localhost:9092", - "reader.option.kafka.security.protocol=PLAINTEXT" - ) - ), - order = 1 + val jobParameters = SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "reader.option.kafka.security.protocol=PLAINTEXT" + ) ) - val result = underTest.getKafkaParameters(jobDefinition) + val result = underTest.getKafkaParameters(jobParameters) result.isDefined shouldBe true result.get._1 shouldBe "some-topic" @@ -124,67 +112,51 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers } it should "return None if the topic is not in the app arguments" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.brokers=http://localhost:9092") - ), - order = 1 + val jobParameters = SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.brokers=http://localhost:9092") ) - val result = underTest.getKafkaParameters(jobDefinition) + val result = underTest.getKafkaParameters(jobParameters) result shouldBe None } it should "return None if the kafka brokers are not in the app arguments" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.topic=some-topic") - ), - order = 1 + val jobParameters = SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.topic=some-topic") ) - val result = underTest.getKafkaParameters(jobDefinition) + val result = underTest.getKafkaParameters(jobParameters) result shouldBe None } it should "return None if the job type is not Hyperdrive" in { - val jobDefinition = ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Spark, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.topic=some-topic", "reader.kafka.brokers=http://localhost:9092") - ), - order = 1 + val jobParameters = SparkInstanceParameters( + jobType = JobTypes.Spark, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List("reader.kafka.topic=some-topic", "reader.kafka.brokers=http://localhost:9092") ) - val result = underTest.getKafkaParameters(jobDefinition) + val result = underTest.getKafkaParameters(jobParameters) result shouldBe None } it should "return None if the jobParameters are not SparkInstanceParameters" in { - val jobDefinition = ResolvedJobDefinition( - "inconsistentJob", - ShellInstanceParameters( - jobType = JobTypes.Hyperdrive, - scriptLocation = "script.sh" - ), - order = 1 + val jobParameters = ShellInstanceParameters( + jobType = JobTypes.Hyperdrive, + scriptLocation = "script.sh" ) - val result = underTest.getKafkaParameters(jobDefinition) + val result = underTest.getKafkaParameters(jobParameters) result shouldBe None } @@ -192,7 +164,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers "isNewJobInstanceRequired" should "return false if the kafka and checkpoint folder offsets are the same" in { val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = getJobDefinition + val jobParameters = getJobParameters when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("/checkpoint/path/some-topic/offsets/21", true))) when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) @@ -200,7 +172,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => val hdfsParametersCaptor: ArgumentCaptor[HdfsParameters] = ArgumentCaptor.forClass(classOf[HdfsParameters]) @@ -217,12 +189,12 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return false if the kafka topic does not exist" in { val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = getJobDefinition + val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map[Int, Long]()) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map[Int, Long]()) - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService, never()).getLatestOffsetFilePath(any()) @@ -233,12 +205,12 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return false if the kafka topic is empty" in { val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = getJobDefinition + val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 42L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 42L)) - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService, never()).getLatestOffsetFilePath(any()) @@ -249,13 +221,13 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return true if no offset file is present" in { val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = getJobDefinition + val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(None) - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) @@ -266,13 +238,13 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return true if the offset is not committed" in { val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = getJobDefinition + val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", false))) - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) @@ -283,7 +255,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return true if a offset file could not be parsed" in { val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = getJobDefinition + val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) @@ -291,7 +263,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) .thenThrow(new RuntimeException("Failed to parse")) - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) @@ -303,7 +275,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return true if the checkpoints offset does not contain the topic" in { val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = getJobDefinition + val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) @@ -311,7 +283,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) .thenReturn(Option(Map("some-other-topic" -> Map(0 -> 21L)))) - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) @@ -323,7 +295,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return true if the kafka offsets and checkpoint offset do not have the same set of partitions" in { val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = getJobDefinition + val jobParameters = getJobParameters when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) @@ -331,7 +303,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) @@ -344,7 +316,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return true if the kafka offsets and checkpoint offsets are not the same" in { val config = getSparkConfig val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobDefinition = getJobDefinition + val jobParameters = getJobParameters when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) @@ -352,7 +324,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 42L, 1 -> 7L)) - val resultFut = underTest.isNewJobInstanceRequired(jobDefinition) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) @@ -370,20 +342,16 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers ) ) - private def getJobDefinition = { - ResolvedJobDefinition( - "job", - SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=some-topic", - "reader.kafka.brokers=http://localhost:9092", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ), - order = 1 + private def getJobParameters = { + SparkInstanceParameters( + jobType = JobTypes.Hyperdrive, + jobJar = "job.jar", + mainClass = "mainClass", + appArguments = List( + "reader.kafka.topic=some-topic", + "reader.kafka.brokers=http://localhost:9092", + "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" + ) ) } } From f2e7f45f06dbe397b02e31b5f7584c8b7b655efa Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 15 Jul 2022 11:56:16 +0200 Subject: [PATCH 21/34] PR fix: Limit number of kafka consumers --- README.md | 4 ++++ src/main/resources/application.properties | 3 +++ .../api/rest/services/KafkaService.scala | 20 ++++++++----------- .../application/GeneralConfig.scala | 4 +++- .../api/rest/services/KafkaServiceTest.scala | 13 ++++-------- .../application/TestGeneralConfig.scala | 5 +++-- 6 files changed, 25 insertions(+), 24 deletions(-) diff --git a/README.md b/README.md index b7d1b7d74..a62ade20e 100644 --- a/README.md +++ b/README.md @@ -130,6 +130,10 @@ spring.mail.host= spring.mail.port= ``` ``` +# Kafka Service properties. Used for per-workflow Kafka consumers +kafka.consumers.cache.size=50 +``` +``` #Kafka sensor properties. Not all are required. Adjust according to your use case. kafkaSource.group.id.prefix=hyper_drive_${appUniqueId} kafkaSource.poll.duration=500 diff --git a/src/main/resources/application.properties b/src/main/resources/application.properties index 171c91f25..55c84d561 100644 --- a/src/main/resources/application.properties +++ b/src/main/resources/application.properties @@ -70,6 +70,9 @@ notification.sender.address= spring.mail.host= spring.mail.port= +# Kafka service properties. +kafka.consumers.cache.size=50 + #Kafka sensor properties. kafkaSource.group.id.prefix=hyper_drive kafkaSource.poll.duration=500 diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index 685fb9a2f..845503e25 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -18,11 +18,12 @@ package za.co.absa.hyperdrive.trigger.api.rest.services import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.common.TopicPartition import org.springframework.stereotype.Service +import org.springframework.util.ConcurrentLruCache import za.co.absa.hyperdrive.trigger.api.rest.services.KafkaServiceImpl.{BeginningOffsets, EndOffsets, OffsetFunction} -import za.co.absa.hyperdrive.trigger.configuration.application.{GeneralConfig, KafkaConfig} +import za.co.absa.hyperdrive.trigger.configuration.application.GeneralConfig import java.util.Properties -import java.util.concurrent.ConcurrentHashMap +import java.util.UUID.randomUUID import javax.inject.Inject import scala.collection.JavaConverters._ @@ -32,9 +33,10 @@ trait KafkaService { } @Service -class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) extends KafkaService { +class KafkaServiceImpl @Inject() (generalConfig: GeneralConfig) extends KafkaService { - private val kafkaConsumersCache = new ConcurrentHashMap[Properties, KafkaConsumer[String, String]]() + private val kafkaConsumersCache = new ConcurrentLruCache[Properties, KafkaConsumer[String, String]]( + generalConfig.kafkaConsumersCacheSize, createKafkaConsumer) override def getBeginningOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] = { getOffsets(topic, consumerProperties, BeginningOffsets) @@ -47,15 +49,9 @@ class KafkaServiceImpl @Inject() (kafkaConfig: KafkaConfig, generalConfig: Gener def createKafkaConsumer(properties: Properties) = new KafkaConsumer[String, String](properties) private def getOffsets(topic: String, properties: Properties, offsetFn: OffsetFunction): Map[Int, Long] = { - val groupId = s"${kafkaConfig.groupIdPrefix}-${generalConfig.appUniqueId}-kafkaService" + val groupId = s"hyperdrive-trigger-kafkaService-${randomUUID().toString}" properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) - val consumer = kafkaConsumersCache.asScala - .getOrElse(properties, { - val consumer = createKafkaConsumer(properties) - kafkaConsumersCache.put(properties, consumer) - consumer - } - ) + val consumer = kafkaConsumersCache.get(properties) val partitionInfo = Option(consumer.partitionsFor(topic)).map(_.asScala).getOrElse(Seq()) val topicPartitions = partitionInfo.map(p => new TopicPartition(p.topic(), p.partition())) val offsets = offsetFn match { diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/configuration/application/GeneralConfig.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/configuration/application/GeneralConfig.scala index 75400bf51..3253f65a3 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/configuration/application/GeneralConfig.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/configuration/application/GeneralConfig.scala @@ -34,5 +34,7 @@ class GeneralConfig( @DefaultValue(Array("Unknown")) val version: String, @(NotBlank @field) - val appUniqueId: String + val appUniqueId: String, + @DefaultValue(Array("50")) + val kafkaConsumersCacheSize: Int ) diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala index 126e6db10..ae3f1d670 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala @@ -21,23 +21,18 @@ import org.mockito.ArgumentMatchers.{any, eq => eqTo} import org.mockito.Mockito.when import org.scalatest.mockito.MockitoSugar import org.scalatest.{FlatSpec, Matchers} -import za.co.absa.hyperdrive.trigger.configuration.application.{ - GeneralConfig, - KafkaConfig, - TestGeneralConfig, - TestKafkaConfig -} +import za.co.absa.hyperdrive.trigger.configuration.application.{GeneralConfig, TestGeneralConfig} import java.util.Properties class KafkaServiceTest extends FlatSpec with MockitoSugar with Matchers { private val mockKafkaConsumer = mock[KafkaConsumer[String, String]] - class KafkaServiceTestImpl(kafkaConfig: KafkaConfig, generalConfig: GeneralConfig) - extends KafkaServiceImpl(kafkaConfig, generalConfig) { + class KafkaServiceTestImpl(generalConfig: GeneralConfig) + extends KafkaServiceImpl(generalConfig) { override def createKafkaConsumer(properties: Properties): KafkaConsumer[String, String] = mockKafkaConsumer } - private val underTest = new KafkaServiceTestImpl(TestKafkaConfig(), TestGeneralConfig()) + private val underTest = new KafkaServiceTestImpl(TestGeneralConfig()) "getEndOffsets" should "return a map of end offsets" in { import scala.collection.JavaConverters._ diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/configuration/application/TestGeneralConfig.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/configuration/application/TestGeneralConfig.scala index abfa0b948..e7e3808ef 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/configuration/application/TestGeneralConfig.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/configuration/application/TestGeneralConfig.scala @@ -20,7 +20,8 @@ object TestGeneralConfig { maximumNumberOfWorkflowsInBulkRun: Int = 10, environment: String = "Unknown", version: String = "Unknown", - appUniqueId: String = "20e3f97d-88ac-453c-9524-0166e2c221c5" + appUniqueId: String = "20e3f97d-88ac-453c-9524-0166e2c221c5", + kafkaConsumersCacheSize: Int = 50 ): GeneralConfig = - new GeneralConfig(maximumNumberOfWorkflowsInBulkRun, environment, version, appUniqueId) + new GeneralConfig(maximumNumberOfWorkflowsInBulkRun, environment, version, appUniqueId, kafkaConsumersCacheSize) } From 97d8b89920b16d76c1f2db4dcc8e5f40856591d1 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Fri, 15 Jul 2022 17:16:06 +0200 Subject: [PATCH 22/34] PR fix: Refactor HdfsService -> CheckpointService --- ...sService.scala => CheckpointService.scala} | 69 ++++++++++--------- .../HyperdriveOffsetComparisonService.scala | 8 +-- ...Test.scala => CheckpointServiceTest.scala} | 45 ++++-------- ...yperdriveOffsetComparisonServiceTest.scala | 22 +++--- 4 files changed, 64 insertions(+), 80 deletions(-) rename src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/{HdfsService.scala => CheckpointService.scala} (91%) rename src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/{HdfsServiceTest.scala => CheckpointServiceTest.scala} (78%) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala similarity index 91% rename from src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala rename to src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala index 336799349..939fd9701 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala @@ -27,11 +27,10 @@ import java.nio.charset.StandardCharsets.UTF_8 import javax.inject.Inject import scala.io.Source -trait HdfsService { +trait CheckpointService { type TopicPartitionOffsets = Map[String, Map[Int, Long]] - def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] - def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets + def getOffsetsFromFile(path: String): Option[TopicPartitionOffsets] def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] def loginUserFromKeytab(principal: String, keytab: String): Unit } @@ -43,7 +42,7 @@ class HdfsParameters( ) @Service -class HdfsServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformationWrapper) extends HdfsService { +class CheckpointServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformationWrapper) extends CheckpointService { private val logger = LoggerFactory.getLogger(this.getClass) private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) private val offsetsDirName = "offsets" @@ -66,6 +65,37 @@ class HdfsServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformati } } + override def getOffsetsFromFile(path: String): Option[TopicPartitionOffsets] = { + parseFileAndClose(path, parseKafkaOffsetStream) + } + + /** + * @return an Option of a String, Boolean pair. The string contains the path to the latest offset file, while the + * boolean is true if the offset is committed (i.e. a corresponding commit file exists), and false otherwise. + * None is returned if the offset file does not exist. If the offset file does not exist, the corresponding + * commit file is assumed to also not exist. + */ + override def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] = { + val offsetBatchIdOpt = getLatestOffsetBatchId(params.checkpointLocation) + val offsetFilePath = offsetBatchIdOpt.map { offsetBatchId => + val commitBatchIdOpt = getLatestCommitBatchId(params.checkpointLocation) + val committed = commitBatchIdOpt match { + case Some(commitBatchId) => offsetBatchId == commitBatchId + case None => false + } + val path = new Path(s"${params.checkpointLocation}/${offsetsDirName}/${offsetBatchId}") + (path.toString, committed) + } + if (offsetFilePath.isEmpty) { + logger.debug(s"No offset files exist under checkpoint location ${params.checkpointLocation}") + } + offsetFilePath + } + + override def loginUserFromKeytab(principal: String, keytab: String): Unit = { + userGroupInformationWrapper.loginUserFromKeytab(principal, keytab) + } + /** * @param pathStr path to the file as a string * @param parseFn function that parses the file line by line. Caution: It must materialize the content, @@ -73,7 +103,7 @@ class HdfsServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformati * @tparam R type of the parsed value * @return None if the file doesn't exist, Some with the parsed content */ - override def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] = { + private def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] = { val path = new Path(pathStr) if (fs.exists(path)) { val input = fs.open(path) @@ -98,7 +128,7 @@ class HdfsServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformati * and org.apache.spark.sql.kafka010.JsonUtils * for details on the assumed format */ - override def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets = { + private def parseKafkaOffsetStream(lines: Iterator[String]): TopicPartitionOffsets = { val SERIALIZED_VOID_OFFSET = "-" def parseOffset(value: String): Option[TopicPartitionOffsets] = value match { case SERIALIZED_VOID_OFFSET => None @@ -118,33 +148,6 @@ class HdfsServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformati .head } - /** - * @return an Option of a String, Boolean pair. The string contains the path to the latest offset file, while the - * boolean is true if the offset is committed (i.e. a corresponding commit file exists), and false otherwise. - * None is returned if the offset file does not exist. If the offset file does not exist, the corresponding - * commit file is assumed to also not exist. - */ - override def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] = { - val offsetBatchIdOpt = getLatestOffsetBatchId(params.checkpointLocation) - val offsetFilePath = offsetBatchIdOpt.map { offsetBatchId => - val commitBatchIdOpt = getLatestCommitBatchId(params.checkpointLocation) - val committed = commitBatchIdOpt match { - case Some(commitBatchId) => offsetBatchId == commitBatchId - case None => false - } - val path = new Path(s"${params.checkpointLocation}/${offsetsDirName}/${offsetBatchId}") - (path.toString, committed) - } - if (offsetFilePath.isEmpty) { - logger.debug(s"No offset files exist under checkpoint location ${params.checkpointLocation}") - } - offsetFilePath - } - - override def loginUserFromKeytab(principal: String, keytab: String): Unit = { - userGroupInformationWrapper.loginUserFromKeytab(principal, keytab) - } - private def getLatestCommitBatchId(checkpointDir: String): Option[Long] = { val commitsDir = new Path(s"$checkpointDir/$commitsDirName") getLatestBatchId(commitsDir) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 946345611..66849520b 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -39,7 +39,7 @@ trait HyperdriveOffsetComparisonService { @Service class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, - hdfsService: HdfsService, + checkpointService: CheckpointService, kafkaService: KafkaService ) extends HyperdriveOffsetComparisonService { private val logger = LoggerFactory.getLogger(this.getClass) @@ -182,8 +182,8 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, Future { val latestOffsetOpt = for { hdfsParameters <- hdfsParametersOpt - _ = hdfsService.loginUserFromKeytab(hdfsParameters.principal, hdfsParameters.keytab) - latestOffset <- hdfsService.getLatestOffsetFilePath(hdfsParameters) + _ = checkpointService.loginUserFromKeytab(hdfsParameters.principal, hdfsParameters.keytab) + latestOffset <- checkpointService.getLatestOffsetFilePath(hdfsParameters) } yield { latestOffset } if (latestOffsetOpt.isEmpty || !latestOffsetOpt.get._2) { logger.debug(s"Offset does not exist or is not committed ${latestOffsetOpt}") @@ -196,7 +196,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, case Some(_) if kafkaParametersOpt.isEmpty => Future { None } case Some(latestOffset) => Future { - hdfsService.parseFileAndClose(latestOffset._1, hdfsService.parseKafkaOffsetStream) + checkpointService.getOffsetsFromFile(latestOffset._1) }.recover { case e: Exception => logger.warn(s"Couldn't parse file ${latestOffset._1}", e) None diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala similarity index 78% rename from src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala rename to src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala index 69c9ecaef..d654e78bd 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala @@ -24,10 +24,10 @@ import za.co.absa.commons.io.TempDirectory import java.nio.charset.StandardCharsets import java.nio.file.{Files, Path} -class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { +class CheckpointServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { private val ugiWrapper = mock[UserGroupInformationWrapper] doNothing().when(ugiWrapper).loginUserFromKeytab(any(), any()) - private val underTest = new HdfsServiceImpl(ugiWrapper) + private val underTest = new CheckpointServiceImpl(ugiWrapper) private var baseDir: TempDirectory = _ private var baseDirPath: Path = _ @@ -40,44 +40,33 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo baseDir.delete() } - "parseFileAndClose" should "successfully parse a file" in { - val tmpFile = Files.createTempFile(baseDirPath, "hdfsServiceTest", "") - val text = Seq("1", "2", "3").mkString("\n") - Files.write(tmpFile, text.getBytes(StandardCharsets.UTF_8)) - val parseFn: Iterator[String] => Seq[Int] = it => it.toList.map(_.toInt) - - val result = underTest.parseFileAndClose(tmpFile.toAbsolutePath.toString, parseFn) - - result.isDefined shouldBe true - result.get should contain theSameElementsAs Seq(1, 2, 3) - } - - it should "return None if the file does not exist" in { - val parseFn: Iterator[String] => Seq[Int] = _ => Seq() - - val result = underTest.parseFileAndClose("non-existent", parseFn) + "getOffsetFromFile" should "return None if the file does not exist" in { + val result = underTest.getOffsetsFromFile("non-existent") result shouldBe None } - it should "rethrow an exception if parsing throws an error" in { + it should "throw an exception if parsing throws an error" in { val tmpFile = Files.createTempFile(baseDirPath, "hdfsServiceTest", "") - val parseFn: Iterator[String] => Seq[Int] = _ => throw new Exception() - val result = the[Exception] thrownBy underTest.parseFileAndClose(tmpFile.toAbsolutePath.toString, parseFn) + val result = the[Exception] thrownBy underTest.getOffsetsFromFile(tmpFile.toAbsolutePath.toString) result.getMessage should include(tmpFile.toAbsolutePath.toString) } - "parseKafkaOffsetStream" should "parse an offset file" in { + it should "parse an offset file" in { + val tmpFile = Files.createTempFile(baseDirPath, "hdfsServiceTest", "") val lines = Seq( "v1", raw"""{"batchWatermarkMs":0,"batchTimestampMs":1633360640176}""", raw"""{"my.topic":{"2":2021,"1":1021,"3":3021,"0":21}, "my.other.topic":{"0":0}}""" - ).toIterator + ).mkString("\n") + Files.write(tmpFile, lines.getBytes(StandardCharsets.UTF_8)) - val result = underTest.parseKafkaOffsetStream(lines) + val resultOpt = underTest.getOffsetsFromFile(tmpFile.toAbsolutePath.toString) + resultOpt.isDefined shouldBe true + val result = resultOpt.get result.size shouldBe 2 result.head._1 shouldBe "my.topic" result.head._2 should contain theSameElementsAs Map("2" -> 2021, "1" -> 1021, "3" -> 3021, "0" -> 21) @@ -85,14 +74,6 @@ class HdfsServiceTest extends FlatSpec with Matchers with BeforeAndAfter with Mo result.toSeq(1)._2 should contain theSameElementsAs Map("0" -> 0) } - it should "throw an exception if the file is incomplete" in { - val lines = Seq().toIterator - - val result = the[Exception] thrownBy underTest.parseKafkaOffsetStream(lines) - - result.getMessage shouldBe "Incomplete log file" - } - "getLatestOffsetFile" should "get the latest offset file, and it is committed" in { val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") createOffsetFile(tmpCheckpointDir, 12) diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala index 00cb21ba2..2c3e391ad 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -25,7 +25,7 @@ import za.co.absa.hyperdrive.trigger.models.enums.JobTypes import za.co.absa.hyperdrive.trigger.models.{ShellInstanceParameters, SparkInstanceParameters} class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers with BeforeAndAfter with MockitoSugar { - private val hdfsService = mock[HdfsService] + private val hdfsService = mock[CheckpointService] private val kafkaService = mock[KafkaService] private val underTest = new HyperdriveOffsetComparisonServiceImpl(DefaultTestSparkConfig().yarn, hdfsService, kafkaService) @@ -167,7 +167,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val jobParameters = getJobParameters when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("/checkpoint/path/some-topic/offsets/21", true))) - when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + when(hdfsService.getOffsetsFromFile(any())) .thenReturn(Option(Map("some-topic" -> Map(2 -> 2021L, 0 -> 21L, 1 -> 1021L)))) when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) @@ -180,7 +180,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers hdfsParametersCaptor.getValue.keytab shouldBe "/path/to/keytab" hdfsParametersCaptor.getValue.principal shouldBe "principal" hdfsParametersCaptor.getValue.checkpointLocation shouldBe "/checkpoint/path/some-topic" - verify(hdfsService).parseFileAndClose(eqTo("/checkpoint/path/some-topic/offsets/21"), any()) + verify(hdfsService).getOffsetsFromFile(eqTo("/checkpoint/path/some-topic/offsets/21")) verify(kafkaService).getEndOffsets(eqTo("some-topic"), any()) result shouldBe false } @@ -260,14 +260,14 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + when(hdfsService.getOffsetsFromFile(any())) .thenThrow(new RuntimeException("Failed to parse")) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).parseFileAndClose(any(), any()) + verify(hdfsService).getOffsetsFromFile(any()) result shouldBe true } } @@ -280,14 +280,14 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + when(hdfsService.getOffsetsFromFile(any())) .thenReturn(Option(Map("some-other-topic" -> Map(0 -> 21L)))) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).parseFileAndClose(any(), any()) + verify(hdfsService).getOffsetsFromFile(any()) result shouldBe true } } @@ -298,7 +298,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val jobParameters = getJobParameters when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + when(hdfsService.getOffsetsFromFile(any())) .thenReturn(Option(Map("some-topic" -> Map(0 -> 21L)))) when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) @@ -307,7 +307,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).parseFileAndClose(any(), any()) + verify(hdfsService).getOffsetsFromFile(any()) verify(kafkaService).getEndOffsets(any(), any()) result shouldBe true } @@ -319,7 +319,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val jobParameters = getJobParameters when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.parseFileAndClose(any(), any[Iterator[String] => Map[String, Map[Int, Long]]]())) + when(hdfsService.getOffsetsFromFile(any())) .thenReturn(Option(Map("some-topic" -> Map(0 -> 42L, 1 -> 55L)))) when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 42L, 1 -> 7L)) @@ -328,7 +328,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers resultFut.map { result => verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).parseFileAndClose(any(), any()) + verify(hdfsService).getOffsetsFromFile(any()) verify(kafkaService).getEndOffsets(any(), any()) result shouldBe true } From 6770f856dd6acabdbb1c8d98d670a63128d10dbe Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 18 Jul 2022 09:42:53 +0200 Subject: [PATCH 23/34] PR fix: Make methods private --- .../HyperdriveOffsetComparisonService.scala | 96 ++++--- ...yperdriveOffsetComparisonServiceTest.scala | 241 ++++++------------ 2 files changed, 132 insertions(+), 205 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 66849520b..a89c34a71 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -31,9 +31,6 @@ import javax.inject.Inject import scala.concurrent.{ExecutionContext, Future} trait HyperdriveOffsetComparisonService { - def getResolvedAppArguments(jobParameters: JobInstanceParameters): Option[Map[String, String]] - def getHdfsParameters(resolvedAppArguments: Map[String, String]): Option[HdfsParameters] - def getKafkaParameters(jobParameters: JobInstanceParameters): Option[(String, Properties)] def isNewJobInstanceRequired(jobParameters: JobInstanceParameters)(implicit ec: ExecutionContext): Future[Boolean] } @@ -50,7 +47,51 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private val PropertyDelimiter = "=" private val ListDelimiter = ',' - override def getResolvedAppArguments(jobParameters: JobInstanceParameters): Option[Map[String, String]] = { + + def isNewJobInstanceRequired(jobParameters: JobInstanceParameters)(implicit ec: ExecutionContext): Future[Boolean] = { + val kafkaParametersOpt = getKafkaParameters(jobParameters) + if (kafkaParametersOpt.isEmpty) { + logger.debug(s"Kafka parameters were not found in job definition ${jobParameters}") + } + + val kafkaEndOffsetsOptFut = Future { + kafkaParametersOpt.map { kafkaParameters => + kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) + } + } + + val kafkaBeginningOffsetsOptFut = Future { + kafkaParametersOpt.map { kafkaParameters => + kafkaService.getBeginningOffsets(kafkaParameters._1, kafkaParameters._2) + } + } + + val isNewJobInstanceRequiredFut = kafkaEndOffsetsOptFut.flatMap { kafkaEndOffsetsOpt => + kafkaBeginningOffsetsOptFut.flatMap { kafkaBeginningOffsetsOpt => + (kafkaBeginningOffsetsOpt, kafkaEndOffsetsOpt) match { + case (Some(kafkaBeginningOffsets), Some(kafkaEndOffsets)) => + if (kafkaBeginningOffsets.isEmpty) { // topic does not exist + Future { false } + } else if (offsetsEqual(kafkaBeginningOffsets, kafkaEndOffsets)) { // topic is empty + Future { false } + } else { + getCheckpointOffsets(jobParameters, kafkaParametersOpt).map { + case Some(checkpointOffsets) => !offsetsConsumed(checkpointOffsets, kafkaEndOffsets) + case _ => true + } + } + case _ => Future { true } + } + } + } + + isNewJobInstanceRequiredFut.recover { case e: Exception => + logger.warn("An error occurred while getting offsets", e) + true + } + } + + private def getResolvedAppArguments(jobParameters: JobInstanceParameters): Option[Map[String, String]] = { if (!isHyperdriveJob(jobParameters)) { logger.warn(s"Job Parameters ${jobParameters} is not a Hyperdrive Job!") None @@ -66,7 +107,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } } - override def getHdfsParameters(resolvedAppArguments: Map[String, String]): Option[HdfsParameters] = { + private def getHdfsParameters(resolvedAppArguments: Map[String, String]): Option[HdfsParameters] = { val hdfsParameters = for { keytab <- sparkConfig.yarn.additionalConfs.get("spark.yarn.keytab") principal <- sparkConfig.yarn.additionalConfs.get("spark.yarn.principal") @@ -83,7 +124,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, hdfsParameters } - override def getKafkaParameters(jobParameters: JobInstanceParameters): Option[(String, Properties)] = { + private def getKafkaParameters(jobParameters: JobInstanceParameters): Option[(String, Properties)] = { if (!isHyperdriveJob(jobParameters)) { logger.warn(s"Job Definition ${jobParameters} is not a Hyperdrive Job!") None @@ -127,49 +168,6 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, jobParameters.jobType == JobTypes.Hyperdrive && jobParameters.isInstanceOf[SparkInstanceParameters] - def isNewJobInstanceRequired(jobParameters: JobInstanceParameters)(implicit ec: ExecutionContext): Future[Boolean] = { - val kafkaParametersOpt = getKafkaParameters(jobParameters) - if (kafkaParametersOpt.isEmpty) { - logger.debug(s"Kafka parameters were not found in job definition ${jobParameters}") - } - - val kafkaEndOffsetsOptFut = Future { - kafkaParametersOpt.map { kafkaParameters => - kafkaService.getEndOffsets(kafkaParameters._1, kafkaParameters._2) - } - } - - val kafkaBeginningOffsetsOptFut = Future { - kafkaParametersOpt.map { kafkaParameters => - kafkaService.getBeginningOffsets(kafkaParameters._1, kafkaParameters._2) - } - } - - val isNewJobInstanceRequiredFut = kafkaEndOffsetsOptFut.flatMap { kafkaEndOffsetsOpt => - kafkaBeginningOffsetsOptFut.flatMap { kafkaBeginningOffsetsOpt => - (kafkaBeginningOffsetsOpt, kafkaEndOffsetsOpt) match { - case (Some(kafkaBeginningOffsets), Some(kafkaEndOffsets)) => - if (kafkaBeginningOffsets.isEmpty) { // topic does not exist - Future { false } - } else if (offsetsEqual(kafkaBeginningOffsets, kafkaEndOffsets)) { // topic is empty - Future { false } - } else { - getCheckpointOffsets(jobParameters, kafkaParametersOpt).map { - case Some(checkpointOffsets) => !offsetsConsumed(checkpointOffsets, kafkaEndOffsets) - case _ => true - } - } - case _ => Future { true } - } - } - } - - isNewJobInstanceRequiredFut.recover { case e: Exception => - logger.warn("An error occurred while getting offsets", e) - true - } - } - private def getCheckpointOffsets(jobParameters: JobInstanceParameters, kafkaParametersOpt: Option[(String, Properties)] )(implicit ec: ExecutionContext): Future[Option[Map[Int, Long]]] = { diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala index 2c3e391ad..69fa2db62 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -25,170 +25,99 @@ import za.co.absa.hyperdrive.trigger.models.enums.JobTypes import za.co.absa.hyperdrive.trigger.models.{ShellInstanceParameters, SparkInstanceParameters} class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers with BeforeAndAfter with MockitoSugar { - private val hdfsService = mock[CheckpointService] + private val checkpointService = mock[CheckpointService] private val kafkaService = mock[KafkaService] private val underTest = - new HyperdriveOffsetComparisonServiceImpl(DefaultTestSparkConfig().yarn, hdfsService, kafkaService) + new HyperdriveOffsetComparisonServiceImpl(DefaultTestSparkConfig().yarn, checkpointService, kafkaService) before { - reset(hdfsService) + reset(checkpointService) reset(kafkaService) } - "getResolvedAppArguments" should "return the resolved app arguments" in { - val jobParameters = SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=my-topic", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ) - val result = underTest.getResolvedAppArguments(jobParameters) - - result.isDefined shouldBe true - result.get should contain theSameElementsAs Map( - "reader.kafka.topic" -> "my-topic", - "writer.common.checkpoint.location" -> "/checkpoint/path/my-topic" - ) - } - - it should "return None if the job type is not hyperdrive" in { - val jobParameters = SparkInstanceParameters( - jobType = JobTypes.Spark, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=my-topic", - "writer.common.checkpoint.location=/checkpoint/path/${reader.kafka.topic}" - ) - ) - - val result = underTest.getResolvedAppArguments(jobParameters) - - result shouldBe None - } - - "getHdfsParameters" should "get the hdfs parameters from the configs" in { - val config = DefaultTestSparkConfig().copy(additionalConfs = - Map( - "spark.yarn.keytab" -> "/path/to/keytab", - "spark.yarn.principal" -> "principal" - ) - ) - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val appArguments = Map( - "writer.common.checkpoint.location" -> "/checkpoint/path/my-topic" - ) - - val params = underTest.getHdfsParameters(appArguments) - - params.isDefined shouldBe true - params.get.keytab shouldBe "/path/to/keytab" - params.get.principal shouldBe "principal" - params.get.checkpointLocation shouldBe "/checkpoint/path/my-topic" - } + "isNewJobInstanceRequired" should "return false if the kafka and checkpoint folder offsets are the same" in { + val config = getSparkConfig + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val jobParameters = getJobParameters - "getKafkaParameters" should "get kafka properties from a resolved job definition" in { - val jobParameters = SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List( - "reader.kafka.topic=some-topic", - "reader.kafka.brokers=http://localhost:9092", - "reader.option.kafka.security.protocol=PLAINTEXT" - ) - ) + when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("/checkpoint/path/some-topic/offsets/21", true))) + when(checkpointService.getOffsetsFromFile(any())) + .thenReturn(Option(Map("some-topic" -> Map(2 -> 2021L, 0 -> 21L, 1 -> 1021L)))) + when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) + when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) - val result = underTest.getKafkaParameters(jobParameters) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) - result.isDefined shouldBe true - result.get._1 shouldBe "some-topic" - val properties = result.get._2 - properties.getProperty("bootstrap.servers") shouldBe "http://localhost:9092" - properties.getProperty("security.protocol") shouldBe "PLAINTEXT" + resultFut.map { result => + val hdfsParametersCaptor: ArgumentCaptor[HdfsParameters] = ArgumentCaptor.forClass(classOf[HdfsParameters]) + verify(checkpointService).getLatestOffsetFilePath(hdfsParametersCaptor.capture()) + hdfsParametersCaptor.getValue.keytab shouldBe "/path/to/keytab" + hdfsParametersCaptor.getValue.principal shouldBe "principal" + hdfsParametersCaptor.getValue.checkpointLocation shouldBe "/checkpoint/path/some-topic" + verify(checkpointService).getOffsetsFromFile(eqTo("/checkpoint/path/some-topic/offsets/21")) + verify(kafkaService).getEndOffsets(eqTo("some-topic"), any()) + result shouldBe false + } } - it should "return None if the topic is not in the app arguments" in { - val jobParameters = SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.brokers=http://localhost:9092") - ) + it should "return true if the job type is not hyperdrive" in { + val jobParameters = getJobParameters.copy(jobType = JobTypes.Spark) - val result = underTest.getKafkaParameters(jobParameters) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) - result shouldBe None + resultFut.map { result => + verify(checkpointService, never()).getLatestOffsetFilePath(any()) + verify(checkpointService, never()).getOffsetsFromFile(any()) + verify(kafkaService, never()).getEndOffsets(any(), any()) + result shouldBe true + } } - it should "return None if the kafka brokers are not in the app arguments" in { - val jobParameters = SparkInstanceParameters( - jobType = JobTypes.Hyperdrive, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.topic=some-topic") - ) + it should "return true if the topic is not in the app arguments" in { + val jobParameters = getJobParameters.copy(appArguments = List("reader.kafka.brokers=http://localhost:9092")) - val result = underTest.getKafkaParameters(jobParameters) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) - result shouldBe None + resultFut.map { result => + verify(checkpointService, never()).getLatestOffsetFilePath(any()) + verify(checkpointService, never()).getOffsetsFromFile(any()) + verify(kafkaService, never()).getEndOffsets(any(), any()) + result shouldBe true + } } - it should "return None if the job type is not Hyperdrive" in { - val jobParameters = SparkInstanceParameters( - jobType = JobTypes.Spark, - jobJar = "job.jar", - mainClass = "mainClass", - appArguments = List("reader.kafka.topic=some-topic", "reader.kafka.brokers=http://localhost:9092") - ) + it should "return true if the kafka brokers are not in the app arguments" in { + val jobParameters = getJobParameters.copy(appArguments = List("reader.kafka.topic=my-topic")) - val result = underTest.getKafkaParameters(jobParameters) + val resultFut = underTest.isNewJobInstanceRequired(jobParameters) - result shouldBe None + resultFut.map { result => + verify(checkpointService, never()).getLatestOffsetFilePath(any()) + verify(checkpointService, never()).getOffsetsFromFile(any()) + verify(kafkaService, never()).getEndOffsets(any(), any()) + result shouldBe true + } } - it should "return None if the jobParameters are not SparkInstanceParameters" in { + it should "return true if the jobParameters are not SparkInstanceParameters" in { val jobParameters = ShellInstanceParameters( jobType = JobTypes.Hyperdrive, scriptLocation = "script.sh" ) - val result = underTest.getKafkaParameters(jobParameters) - - result shouldBe None - } - - "isNewJobInstanceRequired" should "return false if the kafka and checkpoint folder offsets are the same" in { - val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) - val jobParameters = getJobParameters - - when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("/checkpoint/path/some-topic/offsets/21", true))) - when(hdfsService.getOffsetsFromFile(any())) - .thenReturn(Option(Map("some-topic" -> Map(2 -> 2021L, 0 -> 21L, 1 -> 1021L)))) - when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) - when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) - val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - val hdfsParametersCaptor: ArgumentCaptor[HdfsParameters] = ArgumentCaptor.forClass(classOf[HdfsParameters]) - verify(hdfsService).getLatestOffsetFilePath(hdfsParametersCaptor.capture()) - hdfsParametersCaptor.getValue.keytab shouldBe "/path/to/keytab" - hdfsParametersCaptor.getValue.principal shouldBe "principal" - hdfsParametersCaptor.getValue.checkpointLocation shouldBe "/checkpoint/path/some-topic" - verify(hdfsService).getOffsetsFromFile(eqTo("/checkpoint/path/some-topic/offsets/21")) - verify(kafkaService).getEndOffsets(eqTo("some-topic"), any()) - result shouldBe false + verify(checkpointService, never()).getLatestOffsetFilePath(any()) + verify(checkpointService, never()).getOffsetsFromFile(any()) + verify(kafkaService, never()).getEndOffsets(any(), any()) + result shouldBe true } } it should "return false if the kafka topic does not exist" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map[Int, Long]()) @@ -197,14 +126,14 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(hdfsService, never()).getLatestOffsetFilePath(any()) + verify(checkpointService, never()).getLatestOffsetFilePath(any()) result shouldBe false } } it should "return false if the kafka topic is empty" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 42L)) @@ -213,92 +142,92 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(hdfsService, never()).getLatestOffsetFilePath(any()) + verify(checkpointService, never()).getLatestOffsetFilePath(any()) result shouldBe false } } it should "return true if no offset file is present" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) - when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(None) + when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(None) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(hdfsService).getLatestOffsetFilePath(any()) + verify(checkpointService).getLatestOffsetFilePath(any()) result shouldBe true } } it should "return true if the offset is not committed" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) - when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", false))) + when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", false))) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(hdfsService).getLatestOffsetFilePath(any()) + verify(checkpointService).getLatestOffsetFilePath(any()) result shouldBe true } } it should "return true if a offset file could not be parsed" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) - when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.getOffsetsFromFile(any())) + when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) + when(checkpointService.getOffsetsFromFile(any())) .thenThrow(new RuntimeException("Failed to parse")) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).getOffsetsFromFile(any()) + verify(checkpointService).getLatestOffsetFilePath(any()) + verify(checkpointService).getOffsetsFromFile(any()) result shouldBe true } } it should "return true if the checkpoints offset does not contain the topic" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) - when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.getOffsetsFromFile(any())) + when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) + when(checkpointService.getOffsetsFromFile(any())) .thenReturn(Option(Map("some-other-topic" -> Map(0 -> 21L)))) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).getOffsetsFromFile(any()) + verify(checkpointService).getLatestOffsetFilePath(any()) + verify(checkpointService).getOffsetsFromFile(any()) result shouldBe true } } it should "return true if the kafka offsets and checkpoint offset do not have the same set of partitions" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) val jobParameters = getJobParameters - when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.getOffsetsFromFile(any())) + when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) + when(checkpointService.getOffsetsFromFile(any())) .thenReturn(Option(Map("some-topic" -> Map(0 -> 21L)))) when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) @@ -306,8 +235,8 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).getOffsetsFromFile(any()) + verify(checkpointService).getLatestOffsetFilePath(any()) + verify(checkpointService).getOffsetsFromFile(any()) verify(kafkaService).getEndOffsets(any(), any()) result shouldBe true } @@ -315,11 +244,11 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return true if the kafka offsets and checkpoint offsets are not the same" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, hdfsService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) val jobParameters = getJobParameters - when(hdfsService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(hdfsService.getOffsetsFromFile(any())) + when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) + when(checkpointService.getOffsetsFromFile(any())) .thenReturn(Option(Map("some-topic" -> Map(0 -> 42L, 1 -> 55L)))) when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 42L, 1 -> 7L)) @@ -327,8 +256,8 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(hdfsService).getLatestOffsetFilePath(any()) - verify(hdfsService).getOffsetsFromFile(any()) + verify(checkpointService).getLatestOffsetFilePath(any()) + verify(checkpointService).getOffsetsFromFile(any()) verify(kafkaService).getEndOffsets(any(), any()) result shouldBe true } From 013e825f1416020846f8bf2a4c4b7a38e7d80853 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 18 Jul 2022 09:54:01 +0200 Subject: [PATCH 24/34] Add comments / logging --- .../HyperdriveOffsetComparisonService.scala | 24 +++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index a89c34a71..8ec2122f3 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -47,7 +47,18 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private val PropertyDelimiter = "=" private val ListDelimiter = ',' - + /** + * + * @param jobParameters Parameters for the job instance. Should contain at least + * - reader.kafka.topic + * - reader.kafka.brokers + * - writer.common.checkpoint.location + * @param ec ExecutionContext + * @return - false if the job instance can be skipped. This is determined if the checkpoint offset is equal + * to the latest offsets on the kafka topic. A job instance can also be skipped if the kafka topic doesn't exist + * or if it's empty. + * - true otherwise + */ def isNewJobInstanceRequired(jobParameters: JobInstanceParameters)(implicit ec: ExecutionContext): Future[Boolean] = { val kafkaParametersOpt = getKafkaParameters(jobParameters) if (kafkaParametersOpt.isEmpty) { @@ -70,13 +81,18 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, kafkaBeginningOffsetsOptFut.flatMap { kafkaBeginningOffsetsOpt => (kafkaBeginningOffsetsOpt, kafkaEndOffsetsOpt) match { case (Some(kafkaBeginningOffsets), Some(kafkaEndOffsets)) => - if (kafkaBeginningOffsets.isEmpty) { // topic does not exist + if (kafkaBeginningOffsets.isEmpty) { + logger.info(s"Topic ${kafkaParametersOpt.get._1} does not exist. Skipping job instance" ) Future { false } - } else if (offsetsEqual(kafkaBeginningOffsets, kafkaEndOffsets)) { // topic is empty + } else if (offsetsEqual(kafkaBeginningOffsets, kafkaEndOffsets)) { + logger.info(s"Topic ${kafkaParametersOpt.get._1} is empty. Skipping job instance") Future { false } } else { getCheckpointOffsets(jobParameters, kafkaParametersOpt).map { - case Some(checkpointOffsets) => !offsetsConsumed(checkpointOffsets, kafkaEndOffsets) + case Some(checkpointOffsets) => + val allConsumed = offsetsConsumed(checkpointOffsets, kafkaEndOffsets) + logger.info(s"All offsets consumed for topic ${kafkaParametersOpt.get._1}. Skipping job instance") + !allConsumed case _ => true } } From 70a801317cef158c00a10ba404db2a2064befc4a Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 18 Jul 2022 09:55:00 +0200 Subject: [PATCH 25/34] Fix formatting --- .../trigger/api/rest/services/CheckpointService.scala | 3 ++- .../services/HyperdriveOffsetComparisonService.scala | 11 +++++------ .../trigger/api/rest/services/KafkaService.scala | 4 +++- .../HyperdriveOffsetComparisonServiceTest.scala | 3 ++- .../trigger/api/rest/services/KafkaServiceTest.scala | 3 +-- 5 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala index 939fd9701..466c0399d 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala @@ -42,7 +42,8 @@ class HdfsParameters( ) @Service -class CheckpointServiceImpl @Inject()(userGroupInformationWrapper: UserGroupInformationWrapper) extends CheckpointService { +class CheckpointServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformationWrapper) + extends CheckpointService { private val logger = LoggerFactory.getLogger(this.getClass) private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) private val offsetsDirName = "offsets" diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 8ec2122f3..9e0503a76 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -48,13 +48,12 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private val ListDelimiter = ',' /** - * - * @param jobParameters Parameters for the job instance. Should contain at least + * @param jobParameters Parameters for the job instance. Should contain at least * - reader.kafka.topic * - reader.kafka.brokers * - writer.common.checkpoint.location - * @param ec ExecutionContext - * @return - false if the job instance can be skipped. This is determined if the checkpoint offset is equal + * @param ec ExecutionContext + * @return - false if the job instance can be skipped. This is determined if the checkpoint offset is equal * to the latest offsets on the kafka topic. A job instance can also be skipped if the kafka topic doesn't exist * or if it's empty. * - true otherwise @@ -82,7 +81,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, (kafkaBeginningOffsetsOpt, kafkaEndOffsetsOpt) match { case (Some(kafkaBeginningOffsets), Some(kafkaEndOffsets)) => if (kafkaBeginningOffsets.isEmpty) { - logger.info(s"Topic ${kafkaParametersOpt.get._1} does not exist. Skipping job instance" ) + logger.info(s"Topic ${kafkaParametersOpt.get._1} does not exist. Skipping job instance") Future { false } } else if (offsetsEqual(kafkaBeginningOffsets, kafkaEndOffsets)) { logger.info(s"Topic ${kafkaParametersOpt.get._1} is empty. Skipping job instance") @@ -93,7 +92,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, val allConsumed = offsetsConsumed(checkpointOffsets, kafkaEndOffsets) logger.info(s"All offsets consumed for topic ${kafkaParametersOpt.get._1}. Skipping job instance") !allConsumed - case _ => true + case _ => true } } case _ => Future { true } diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index 845503e25..bc4dea7d8 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -36,7 +36,9 @@ trait KafkaService { class KafkaServiceImpl @Inject() (generalConfig: GeneralConfig) extends KafkaService { private val kafkaConsumersCache = new ConcurrentLruCache[Properties, KafkaConsumer[String, String]]( - generalConfig.kafkaConsumersCacheSize, createKafkaConsumer) + generalConfig.kafkaConsumersCacheSize, + createKafkaConsumer + ) override def getBeginningOffsets(topic: String, consumerProperties: Properties): Map[Int, Long] = { getOffsets(topic, consumerProperties, BeginningOffsets) diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala index 69fa2db62..bee6e2e15 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -40,7 +40,8 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) val jobParameters = getJobParameters - when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("/checkpoint/path/some-topic/offsets/21", true))) + when(checkpointService.getLatestOffsetFilePath(any())) + .thenReturn(Some(("/checkpoint/path/some-topic/offsets/21", true))) when(checkpointService.getOffsetsFromFile(any())) .thenReturn(Option(Map("some-topic" -> Map(2 -> 2021L, 0 -> 21L, 1 -> 1021L)))) when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala index ae3f1d670..38764569a 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala @@ -28,8 +28,7 @@ import java.util.Properties class KafkaServiceTest extends FlatSpec with MockitoSugar with Matchers { private val mockKafkaConsumer = mock[KafkaConsumer[String, String]] - class KafkaServiceTestImpl(generalConfig: GeneralConfig) - extends KafkaServiceImpl(generalConfig) { + class KafkaServiceTestImpl(generalConfig: GeneralConfig) extends KafkaServiceImpl(generalConfig) { override def createKafkaConsumer(properties: Properties): KafkaConsumer[String, String] = mockKafkaConsumer } private val underTest = new KafkaServiceTestImpl(TestGeneralConfig()) From c7f8309dd34e8f944fc672f157194b754c0f01a6 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 18 Jul 2022 16:11:47 +0200 Subject: [PATCH 26/34] Make kafka consumer cache per thread --- .../hyperdrive/trigger/api/rest/services/KafkaService.scala | 6 +++--- .../trigger/api/rest/services/KafkaServiceTest.scala | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index bc4dea7d8..f3a9290ae 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -35,7 +35,7 @@ trait KafkaService { @Service class KafkaServiceImpl @Inject() (generalConfig: GeneralConfig) extends KafkaService { - private val kafkaConsumersCache = new ConcurrentLruCache[Properties, KafkaConsumer[String, String]]( + private val kafkaConsumersCache = new ConcurrentLruCache[(Properties, Long), KafkaConsumer[String, String]]( generalConfig.kafkaConsumersCacheSize, createKafkaConsumer ) @@ -48,12 +48,12 @@ class KafkaServiceImpl @Inject() (generalConfig: GeneralConfig) extends KafkaSer getOffsets(topic, consumerProperties, EndOffsets) } - def createKafkaConsumer(properties: Properties) = new KafkaConsumer[String, String](properties) + def createKafkaConsumer(propertiesThreadId: (Properties, Long)) = new KafkaConsumer[String, String](propertiesThreadId._1) private def getOffsets(topic: String, properties: Properties, offsetFn: OffsetFunction): Map[Int, Long] = { val groupId = s"hyperdrive-trigger-kafkaService-${randomUUID().toString}" properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) - val consumer = kafkaConsumersCache.get(properties) + val consumer = kafkaConsumersCache.get((properties, Thread.currentThread().getId)) val partitionInfo = Option(consumer.partitionsFor(topic)).map(_.asScala).getOrElse(Seq()) val topicPartitions = partitionInfo.map(p => new TopicPartition(p.topic(), p.partition())) val offsets = offsetFn match { diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala index 38764569a..b4509f2a3 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala @@ -29,7 +29,7 @@ class KafkaServiceTest extends FlatSpec with MockitoSugar with Matchers { private val mockKafkaConsumer = mock[KafkaConsumer[String, String]] class KafkaServiceTestImpl(generalConfig: GeneralConfig) extends KafkaServiceImpl(generalConfig) { - override def createKafkaConsumer(properties: Properties): KafkaConsumer[String, String] = mockKafkaConsumer + override def createKafkaConsumer(propertiesThreadId: (Properties, Long)): KafkaConsumer[String, String] = mockKafkaConsumer } private val underTest = new KafkaServiceTestImpl(TestGeneralConfig()) From cd4c7f431fabef3b7b51f358e2bb20e743cd380d Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 18 Jul 2022 16:44:01 +0200 Subject: [PATCH 27/34] Add a default deserializer to read from kafka --- .../api/rest/services/HyperdriveOffsetComparisonService.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 9e0503a76..107d569b0 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -46,6 +46,7 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private val HyperdriveKafkaExtraOptionsKey = "reader.option.kafka" private val PropertyDelimiter = "=" private val ListDelimiter = ',' + private val defaultDeserializer = "org.apache.kafka.common.serialization.StringDeserializer" /** * @param jobParameters Parameters for the job instance. Should contain at least @@ -166,6 +167,8 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, } yield { val properties = new Properties() properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) + properties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, defaultDeserializer) + properties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, defaultDeserializer) extraArgs.foreach { case (key, value) => properties.setProperty(key, value) } (topic, properties) } From 99769c69819ac960e78f4a521eae4792d132c33f Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 18 Jul 2022 16:46:31 +0200 Subject: [PATCH 28/34] fix formatting --- .../trigger/api/rest/services/KafkaServiceTest.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala index b4509f2a3..bbe943ca2 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaServiceTest.scala @@ -29,7 +29,8 @@ class KafkaServiceTest extends FlatSpec with MockitoSugar with Matchers { private val mockKafkaConsumer = mock[KafkaConsumer[String, String]] class KafkaServiceTestImpl(generalConfig: GeneralConfig) extends KafkaServiceImpl(generalConfig) { - override def createKafkaConsumer(propertiesThreadId: (Properties, Long)): KafkaConsumer[String, String] = mockKafkaConsumer + override def createKafkaConsumer(propertiesThreadId: (Properties, Long)): KafkaConsumer[String, String] = + mockKafkaConsumer } private val underTest = new KafkaServiceTestImpl(TestGeneralConfig()) From 291e832f6cff8db391a6b26d8a13b1f7a12b0b73 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Mon, 18 Jul 2022 16:52:46 +0200 Subject: [PATCH 29/34] fix formatting 2 --- .../hyperdrive/trigger/api/rest/services/KafkaService.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index f3a9290ae..50b1505c2 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -48,7 +48,8 @@ class KafkaServiceImpl @Inject() (generalConfig: GeneralConfig) extends KafkaSer getOffsets(topic, consumerProperties, EndOffsets) } - def createKafkaConsumer(propertiesThreadId: (Properties, Long)) = new KafkaConsumer[String, String](propertiesThreadId._1) + def createKafkaConsumer(propertiesThreadId: (Properties, Long)) = + new KafkaConsumer[String, String](propertiesThreadId._1) private def getOffsets(topic: String, properties: Properties, offsetFn: OffsetFunction): Map[Int, Long] = { val groupId = s"hyperdrive-trigger-kafkaService-${randomUUID().toString}" From c62bdaab4cb0a56a470278185f25d2426a32377b Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Wed, 20 Jul 2022 19:35:55 +0200 Subject: [PATCH 30/34] Add HdfsService, additional logging for kafka consumer --- .../api/rest/services/CheckpointService.scala | 116 ++++++----- .../api/rest/services/HdfsService.scala | 70 +++++++ .../HyperdriveOffsetComparisonService.scala | 33 +-- .../api/rest/services/KafkaService.scala | 10 +- ...cala => UserGroupInformationService.scala} | 10 +- .../trigger/api/rest/utils/ScalaUtil.scala | 29 +++ .../rest/services/CheckpointServiceTest.scala | 193 +++++++++++------- ...yperdriveOffsetComparisonServiceTest.scala | 109 +++++----- 8 files changed, 373 insertions(+), 197 deletions(-) create mode 100644 src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala rename src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/{UserGroupInformationWrapper.scala => UserGroupInformationService.scala} (73%) create mode 100644 src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtil.scala diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala index 466c0399d..b4db2d1ca 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala @@ -19,20 +19,24 @@ import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.apache.commons.io.IOUtils import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} +import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.deploy.SparkHadoopUtil import org.slf4j.LoggerFactory import org.springframework.stereotype.Service +import za.co.absa.hyperdrive.trigger.api.rest.utils.ScalaUtil.swap import java.nio.charset.StandardCharsets.UTF_8 import javax.inject.Inject import scala.io.Source +import scala.util.Try trait CheckpointService { type TopicPartitionOffsets = Map[String, Map[Int, Long]] - def getOffsetsFromFile(path: String): Option[TopicPartitionOffsets] - def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] - def loginUserFromKeytab(principal: String, keytab: String): Unit + def getOffsetsFromFile(path: String)(implicit ugi: UserGroupInformation): Try[Option[TopicPartitionOffsets]] + def getLatestOffsetFilePath(params: HdfsParameters)(implicit + ugi: UserGroupInformation + ): Try[Option[(String, Boolean)]] } class HdfsParameters( @@ -42,8 +46,7 @@ class HdfsParameters( ) @Service -class CheckpointServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInformationWrapper) - extends CheckpointService { +class CheckpointServiceImpl @Inject() (hdfsService: HdfsService) extends CheckpointService { private val logger = LoggerFactory.getLogger(this.getClass) private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) private val offsetsDirName = "offsets" @@ -66,7 +69,9 @@ class CheckpointServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInf } } - override def getOffsetsFromFile(path: String): Option[TopicPartitionOffsets] = { + override def getOffsetsFromFile( + path: String + )(implicit ugi: UserGroupInformation): Try[Option[TopicPartitionOffsets]] = { parseFileAndClose(path, parseKafkaOffsetStream) } @@ -76,25 +81,25 @@ class CheckpointServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInf * None is returned if the offset file does not exist. If the offset file does not exist, the corresponding * commit file is assumed to also not exist. */ - override def getLatestOffsetFilePath(params: HdfsParameters): Option[(String, Boolean)] = { - val offsetBatchIdOpt = getLatestOffsetBatchId(params.checkpointLocation) - val offsetFilePath = offsetBatchIdOpt.map { offsetBatchId => - val commitBatchIdOpt = getLatestCommitBatchId(params.checkpointLocation) - val committed = commitBatchIdOpt match { - case Some(commitBatchId) => offsetBatchId == commitBatchId - case None => false + override def getLatestOffsetFilePath( + params: HdfsParameters + )(implicit ugi: UserGroupInformation): Try[Option[(String, Boolean)]] = { + getLatestOffsetBatchId(params.checkpointLocation).flatMap { offsetBatchIdOpt => + val offsetFilePath = offsetBatchIdOpt.map { offsetBatchId => + getLatestCommitBatchId(params.checkpointLocation).map { commitBatchIdOpt => + val committed = commitBatchIdOpt match { + case Some(commitBatchId) => offsetBatchId == commitBatchId + case None => false + } + val path = new Path(s"${params.checkpointLocation}/${offsetsDirName}/${offsetBatchId}") + (path.toString, committed) + } } - val path = new Path(s"${params.checkpointLocation}/${offsetsDirName}/${offsetBatchId}") - (path.toString, committed) - } - if (offsetFilePath.isEmpty) { - logger.debug(s"No offset files exist under checkpoint location ${params.checkpointLocation}") + if (offsetFilePath.isEmpty) { + logger.debug(s"No offset files exist under checkpoint location ${params.checkpointLocation}") + } + swap(offsetFilePath) } - offsetFilePath - } - - override def loginUserFromKeytab(principal: String, keytab: String): Unit = { - userGroupInformationWrapper.loginUserFromKeytab(principal, keytab) } /** @@ -104,23 +109,29 @@ class CheckpointServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInf * @tparam R type of the parsed value * @return None if the file doesn't exist, Some with the parsed content */ - private def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R): Option[R] = { + private def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R)(implicit + ugi: UserGroupInformation + ): Try[Option[R]] = { val path = new Path(pathStr) - if (fs.exists(path)) { - val input = fs.open(path) - try { - val lines = Source.fromInputStream(input, UTF_8.name()).getLines() - Some(parseFn(lines)) - } catch { - case e: Exception => - // re-throw the exception with the log file path added - throw new Exception(s"Failed to parse file $path", e) - } finally { - IOUtils.closeQuietly(input) + hdfsService.exists(path).flatMap { exists => + if (exists) { + hdfsService.open(path).map { input => + try { + val lines = Source.fromInputStream(input, UTF_8.name()).getLines() + Some(parseFn(lines)) + } catch { + case e: Exception => + // re-throw the exception with the log file path added + throw new Exception(s"Failed to parse file $path", e) + } finally { + IOUtils.closeQuietly(input) + } + } + + } else { + logger.debug(s"Could not find file $path") + Try(None) } - } else { - logger.debug(s"Could not find file $path") - None } } @@ -149,27 +160,32 @@ class CheckpointServiceImpl @Inject() (userGroupInformationWrapper: UserGroupInf .head } - private def getLatestCommitBatchId(checkpointDir: String): Option[Long] = { + private def getLatestCommitBatchId(checkpointDir: String)(implicit ugi: UserGroupInformation): Try[Option[Long]] = { val commitsDir = new Path(s"$checkpointDir/$commitsDirName") getLatestBatchId(commitsDir) } - private def getLatestOffsetBatchId(checkpointDir: String): Option[Long] = { + private def getLatestOffsetBatchId(checkpointDir: String)(implicit ugi: UserGroupInformation): Try[Option[Long]] = { val offsetsDir = new Path(s"$checkpointDir/$offsetsDirName") getLatestBatchId(offsetsDir) } - private def getLatestBatchId(path: Path): Option[Long] = { - if (fs.exists(path)) { - fs.listStatus(path, batchFilesFilter) - .map { status => - status.getPath.getName.toLong + private def getLatestBatchId(path: Path)(implicit ugi: UserGroupInformation): Try[Option[Long]] = { + hdfsService.exists(path).flatMap { exists => + if (exists) { + hdfsService.listStatus(path, batchFilesFilter).map { statuses => + statuses + .map { status => + status.getPath.getName.toLong + } + .sorted + .lastOption + } - .sorted - .lastOption - } else { - logger.debug(s"Could not find path $path") - None + } else { + logger.debug(s"Could not find path $path") + Try(None) + } } } } diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala new file mode 100644 index 000000000..38a1d1c16 --- /dev/null +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -0,0 +1,70 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.apache.hadoop.fs._ +import org.apache.hadoop.security.UserGroupInformation +import org.apache.spark.deploy.SparkHadoopUtil +import org.springframework.stereotype.Service + +import java.security.PrivilegedExceptionAction +import scala.util.Try + +trait HdfsService { + def exists(path: Path)(implicit ugi: UserGroupInformation): Try[Boolean] + def open(path: Path)(implicit ugi: UserGroupInformation): Try[FSDataInputStream] + def listStatus(path: Path, filter: PathFilter)(implicit ugi: UserGroupInformation): Try[Array[FileStatus]] +} + +@Service +class HdfsServiceImpl extends HdfsService { + private lazy val conf = SparkHadoopUtil.get.conf + override def exists(path: Path)(implicit ugi: UserGroupInformation): Try[Boolean] = { + Try { + doAs { + fs.exists(path) + } + } + } + + override def open(path: Path)(implicit ugi: UserGroupInformation): Try[FSDataInputStream] = { + Try { + doAs { + fs.open(path) + } + } + } + + override def listStatus(path: Path, filter: PathFilter)(implicit + ugi: UserGroupInformation + ): Try[Array[FileStatus]] = { + Try { + doAs { + fs.listStatus(path, filter) + } + } + } + + private def fs = FileSystem.get(conf) + + private def doAs[T](fn: => T)(implicit ugi: UserGroupInformation) = { + ugi.doAs(new PrivilegedExceptionAction[T] { + override def run(): T = { + fn + } + }) + } +} diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala index 107d569b0..8b7bed734 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonService.scala @@ -19,6 +19,7 @@ import org.apache.commons.configuration2.builder.BasicConfigurationBuilder import org.apache.commons.configuration2.builder.fluent.Parameters import org.apache.commons.configuration2.convert.DefaultListDelimiterHandler import org.apache.commons.configuration2.{BaseConfiguration, Configuration} +import org.apache.hadoop.security.UserGroupInformation import org.apache.kafka.clients.consumer.ConsumerConfig import org.slf4j.LoggerFactory import org.springframework.stereotype.Service @@ -37,6 +38,7 @@ trait HyperdriveOffsetComparisonService { @Service class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, checkpointService: CheckpointService, + userGroupInformationService: UserGroupInformationService, kafkaService: KafkaService ) extends HyperdriveOffsetComparisonService { private val logger = LoggerFactory.getLogger(this.getClass) @@ -189,34 +191,35 @@ class HyperdriveOffsetComparisonServiceImpl @Inject() (sparkConfig: SparkConfig, private def getCheckpointOffsets(jobParameters: JobInstanceParameters, kafkaParametersOpt: Option[(String, Properties)] )(implicit ec: ExecutionContext): Future[Option[Map[Int, Long]]] = { - val hdfsParametersOpt = getResolvedAppArguments(jobParameters).flatMap(getHdfsParameters) + case class UGIOffset(ugi: UserGroupInformation, latestOffset: (String, Boolean)) + val hdfsParametersOpt: Option[HdfsParameters] = getResolvedAppArguments(jobParameters).flatMap(getHdfsParameters) if (hdfsParametersOpt.isEmpty) { logger.debug(s"Hdfs parameters were not found in job definition ${jobParameters}") } Future { - val latestOffsetOpt = for { + val ugiOffset = for { hdfsParameters <- hdfsParametersOpt - _ = checkpointService.loginUserFromKeytab(hdfsParameters.principal, hdfsParameters.keytab) - latestOffset <- checkpointService.getLatestOffsetFilePath(hdfsParameters) - } yield { latestOffset } - if (latestOffsetOpt.isEmpty || !latestOffsetOpt.get._2) { - logger.debug(s"Offset does not exist or is not committed ${latestOffsetOpt}") + ugi = userGroupInformationService.loginUserFromKeytab(hdfsParameters.principal, hdfsParameters.keytab) + latestOffset <- checkpointService.getLatestOffsetFilePath(hdfsParameters)(ugi).get + } yield { UGIOffset(ugi, latestOffset) } + if (ugiOffset.isEmpty || !ugiOffset.get.latestOffset._2) { + logger.debug(s"Offset does not exist or is not committed ${ugiOffset.map(_.latestOffset)}") None } else { - latestOffsetOpt + ugiOffset } }.flatMap { case None => Future { None } case Some(_) if kafkaParametersOpt.isEmpty => Future { None } - case Some(latestOffset) => - Future { - checkpointService.getOffsetsFromFile(latestOffset._1) - }.recover { case e: Exception => - logger.warn(s"Couldn't parse file ${latestOffset._1}", e) - None - } + case Some(ugiOffset) => + Future + .fromTry(checkpointService.getOffsetsFromFile(ugiOffset.latestOffset._1)(ugiOffset.ugi)) + .recover { case e: Exception => + logger.warn(s"Couldn't parse file ${ugiOffset.latestOffset._1}", e) + None + } }.map { hdfsAllOffsetsOpt => hdfsAllOffsetsOpt.flatMap { hdfsAllOffsets => val kafkaParameters = kafkaParametersOpt.get diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala index 50b1505c2..2a8f1fde4 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/KafkaService.scala @@ -17,6 +17,7 @@ package za.co.absa.hyperdrive.trigger.api.rest.services import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.common.TopicPartition +import org.slf4j.LoggerFactory import org.springframework.stereotype.Service import org.springframework.util.ConcurrentLruCache import za.co.absa.hyperdrive.trigger.api.rest.services.KafkaServiceImpl.{BeginningOffsets, EndOffsets, OffsetFunction} @@ -34,7 +35,7 @@ trait KafkaService { @Service class KafkaServiceImpl @Inject() (generalConfig: GeneralConfig) extends KafkaService { - + private val logger = LoggerFactory.getLogger(this.getClass) private val kafkaConsumersCache = new ConcurrentLruCache[(Properties, Long), KafkaConsumer[String, String]]( generalConfig.kafkaConsumersCacheSize, createKafkaConsumer @@ -48,8 +49,13 @@ class KafkaServiceImpl @Inject() (generalConfig: GeneralConfig) extends KafkaSer getOffsets(topic, consumerProperties, EndOffsets) } - def createKafkaConsumer(propertiesThreadId: (Properties, Long)) = + def createKafkaConsumer(propertiesThreadId: (Properties, Long)): KafkaConsumer[String, String] = { + logger.info( + s"Creating new Kafka Consumer for thread id ${propertiesThreadId._2} and" + + s" properties ${propertiesThreadId._1}. Current cache size is ${kafkaConsumersCache.size()}" + ) new KafkaConsumer[String, String](propertiesThreadId._1) + } private def getOffsets(topic: String, properties: Properties, offsetFn: OffsetFunction): Map[Int, Long] = { val groupId = s"hyperdrive-trigger-kafkaService-${randomUUID().toString}" diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationWrapper.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationService.scala similarity index 73% rename from src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationWrapper.scala rename to src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationService.scala index 60e4e6ba2..7cef73b12 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationWrapper.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/UserGroupInformationService.scala @@ -19,12 +19,12 @@ package za.co.absa.hyperdrive.trigger.api.rest.services import org.apache.hadoop.security.UserGroupInformation import org.springframework.stereotype.Service -trait UserGroupInformationWrapper { - def loginUserFromKeytab(principal: String, keytab: String): Unit +trait UserGroupInformationService { + def loginUserFromKeytab(principal: String, keytab: String): UserGroupInformation } @Service -class UserGroupInformationWrapperImpl extends UserGroupInformationWrapper { - override def loginUserFromKeytab(principal: String, keytab: String): Unit = - UserGroupInformation.loginUserFromKeytab(principal, keytab) +class UserGroupInformationServiceImpl extends UserGroupInformationService { + override def loginUserFromKeytab(principal: String, keytab: String): UserGroupInformation = + UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) } diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtil.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtil.scala new file mode 100644 index 000000000..377ed77ae --- /dev/null +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtil.scala @@ -0,0 +1,29 @@ +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.utils + +import scala.util.Try + +object ScalaUtil { + def swap[T](optTry: Option[Try[T]]): Try[Option[T]] = { + import scala.util.{Failure, Success} + optTry match { + case Some(Success(t)) => Success(Some(t)) + case Some(Failure(e)) => Failure(e) + case None => Success(None) + } + } +} diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala index d654e78bd..b014e9256 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala @@ -15,25 +15,34 @@ package za.co.absa.hyperdrive.trigger.api.rest.services +import org.apache.hadoop.fs +import org.apache.hadoop.fs.{FileStatus, FileSystem} +import org.apache.hadoop.security.UserGroupInformation +import org.apache.spark.deploy.SparkHadoopUtil import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.doNothing +import org.mockito.Mockito.{reset, when} +import org.mockito.invocation.InvocationOnMock import org.scalatest.mockito.MockitoSugar import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} import za.co.absa.commons.io.TempDirectory import java.nio.charset.StandardCharsets import java.nio.file.{Files, Path} +import scala.util.{Failure, Try} class CheckpointServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { - private val ugiWrapper = mock[UserGroupInformationWrapper] - doNothing().when(ugiWrapper).loginUserFromKeytab(any(), any()) - private val underTest = new CheckpointServiceImpl(ugiWrapper) + private val hdfsService = mock[HdfsService] + private val ugi = mock[UserGroupInformation] + private val underTest = new CheckpointServiceImpl(hdfsService) private var baseDir: TempDirectory = _ private var baseDirPath: Path = _ + private lazy val conf = SparkHadoopUtil.get.conf + private val localFs = FileSystem.get(conf) before { baseDir = TempDirectory("HdfsServiceTest").deleteOnExit() baseDirPath = baseDir.path.toAbsolutePath + reset(hdfsService) } after { @@ -41,32 +50,37 @@ class CheckpointServiceTest extends FlatSpec with Matchers with BeforeAndAfter w } "getOffsetFromFile" should "return None if the file does not exist" in { - val result = underTest.getOffsetsFromFile("non-existent") + when(hdfsService.exists(any())(any())).thenReturn(Try(false)) - result shouldBe None + val result = underTest.getOffsetsFromFile("non-existent")(ugi) + + result shouldBe Try(None) } it should "throw an exception if parsing throws an error" in { - val tmpFile = Files.createTempFile(baseDirPath, "hdfsServiceTest", "") + when(hdfsService.exists(any())(any())).thenReturn(Try(true)) + when(hdfsService.open(any())(any())).thenReturn(Try(createInputStream(""))) - val result = the[Exception] thrownBy underTest.getOffsetsFromFile(tmpFile.toAbsolutePath.toString) + val result = underTest.getOffsetsFromFile("some-file")(ugi) - result.getMessage should include(tmpFile.toAbsolutePath.toString) + result.isFailure shouldBe true + result.failed.get.getMessage should include("some-file") } it should "parse an offset file" in { - val tmpFile = Files.createTempFile(baseDirPath, "hdfsServiceTest", "") val lines = Seq( "v1", raw"""{"batchWatermarkMs":0,"batchTimestampMs":1633360640176}""", raw"""{"my.topic":{"2":2021,"1":1021,"3":3021,"0":21}, "my.other.topic":{"0":0}}""" ).mkString("\n") - Files.write(tmpFile, lines.getBytes(StandardCharsets.UTF_8)) + when(hdfsService.exists(any())(any())).thenReturn(Try(true)) + when(hdfsService.open(any())(any())).thenReturn(Try(createInputStream(lines))) - val resultOpt = underTest.getOffsetsFromFile(tmpFile.toAbsolutePath.toString) + val resultTryOpt = underTest.getOffsetsFromFile("some-file")(ugi) - resultOpt.isDefined shouldBe true - val result = resultOpt.get + resultTryOpt.isSuccess shouldBe true + resultTryOpt.get.isDefined shouldBe true + val result = resultTryOpt.get.get result.size shouldBe 2 result.head._1 shouldBe "my.topic" result.head._2 should contain theSameElementsAs Map("2" -> 2021, "1" -> 1021, "3" -> 3021, "0" -> 21) @@ -74,92 +88,119 @@ class CheckpointServiceTest extends FlatSpec with Matchers with BeforeAndAfter w result.toSeq(1)._2 should contain theSameElementsAs Map("0" -> 0) } - "getLatestOffsetFile" should "get the latest offset file, and it is committed" in { - val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - createOffsetFile(tmpCheckpointDir, 12) - createCommitFile(tmpCheckpointDir, 12) - - val params = new HdfsParameters( - keytab = "", - principal = "", - checkpointLocation = tmpCheckpointDir.toAbsolutePath.toString - ) + it should "return Failure if an exception occurred while accessing the file system" in { + when(hdfsService.exists(any())(any())).thenReturn(Failure(new RuntimeException("Failed"))) - val result = underTest.getLatestOffsetFilePath(params) + val result = underTest.getOffsetsFromFile("some-file")(ugi) - result.isDefined shouldBe true - result.get._1 shouldBe s"${tmpCheckpointDir.toAbsolutePath.toString}/offsets/12" - result.get._2 shouldBe true + result.isFailure shouldBe true } - it should "get the latest offset file, and committed = true, if the commits folder is empty" in { - val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - createOffsetFile(tmpCheckpointDir, 12) - Files.createDirectory(tmpCheckpointDir.resolve("commits")) + "getLatestOffsetFile" should "get the latest offset file, and it is committed" in { + when(hdfsService.exists(any())(any())).thenReturn(Try(true)) + when(hdfsService.listStatus(any(), any())(any())).thenReturn(Try(createOffsetFiles(12))) + val params = getHdfsParameters - val params = new HdfsParameters( - keytab = "", - principal = "", - checkpointLocation = tmpCheckpointDir.toAbsolutePath.toString - ) + val resultTryOpt = underTest.getLatestOffsetFilePath(params)(ugi) - val result = underTest.getLatestOffsetFilePath(params) + resultTryOpt.isSuccess shouldBe true + resultTryOpt.get.isDefined shouldBe true + val result = resultTryOpt.get.get + result._1 shouldBe s"/checkpoints/offsets/12" + result._2 shouldBe true + } - result.isDefined shouldBe true - result.get._1 shouldBe s"${tmpCheckpointDir.toAbsolutePath.toString}/offsets/12" - result.get._2 shouldBe false + it should "get the latest offset file, and committed = false, if the commits folder is empty" in { + when(hdfsService.exists(any())(any())).thenReturn(Try(true)) + when(hdfsService.listStatus(any(), any())(any())).thenAnswer((invocation: InvocationOnMock) => { + val path = invocation.getArgument[fs.Path](0) + if (path.toString.contains("offsets")) { + Try(createOffsetFiles(12)) + } else { + Try(Array[FileStatus]()) + } + }) + val params = getHdfsParameters + + val resultTryOpt = underTest.getLatestOffsetFilePath(params)(ugi) + + resultTryOpt.isSuccess shouldBe true + resultTryOpt.get.isDefined shouldBe true + val result = resultTryOpt.get.get + result._1 shouldBe "/checkpoints/offsets/12" + result._2 shouldBe false } it should "get the latest offset file, and it is not committed" in { - val tmpCheckpointDir = Files.createTempDirectory(baseDirPath, "checkpoints") - createOffsetFile(tmpCheckpointDir, 12) - createCommitFile(tmpCheckpointDir, 11) + when(hdfsService.exists(any())(any())).thenReturn(Try(true)) + when(hdfsService.listStatus(any(), any())(any())).thenAnswer((invocation: InvocationOnMock) => { + val path = invocation.getArgument[fs.Path](0) + if (path.toString.contains("offsets")) { + Try(createOffsetFiles(12)) + } else { + Try(createOffsetFiles(11)) + } + }) + val params = getHdfsParameters + + val resultTryOpt = underTest.getLatestOffsetFilePath(params)(ugi) + + resultTryOpt.isSuccess shouldBe true + resultTryOpt.get.isDefined shouldBe true + val result = resultTryOpt.get.get + result._1 shouldBe "/checkpoints/offsets/12" + result._2 shouldBe false + } - val params = new HdfsParameters( - keytab = "", - principal = "", - checkpointLocation = tmpCheckpointDir.toAbsolutePath.toString - ) + it should "return None if the checkpoints folder does not exist" in { + when(hdfsService.exists(any())(any())).thenReturn(Try(false)) + val params = getHdfsParameters - val result = underTest.getLatestOffsetFilePath(params) + val result = underTest.getLatestOffsetFilePath(params)(ugi) - result.isDefined shouldBe true - result.get._1 shouldBe s"${tmpCheckpointDir.toAbsolutePath.toString}/offsets/12" - result.get._2 shouldBe false + result.isSuccess shouldBe true + result.get.isDefined shouldBe false } - it should "return None if the checkpoints folder does not exist" in { - val params = new HdfsParameters( - keytab = "", - principal = "", - checkpointLocation = "non-existent" - ) + it should "return None if the offsets folder is empty" in { + when(hdfsService.exists(any())(any())).thenReturn(Try(true)) + when(hdfsService.listStatus(any(), any())(any())).thenReturn(Try(Array[FileStatus]())) + val params = getHdfsParameters - val result = underTest.getLatestOffsetFilePath(params) + val result = underTest.getLatestOffsetFilePath(params)(ugi) - result.isDefined shouldBe false + result.isSuccess shouldBe true + result.get.isDefined shouldBe false } - it should "return None if the offsets folder is empty" in { - val params = new HdfsParameters( - keytab = "", - principal = "", - checkpointLocation = "checkpoints" - ) - Files.createTempDirectory(baseDirPath, "checkpoints") + it should "return Failure if an exception occurred while accessing the file system" in { + val params = getHdfsParameters + when(hdfsService.exists(any())(any())).thenReturn(Failure(new RuntimeException("Failed"))) - val result = underTest.getLatestOffsetFilePath(params) + val result = underTest.getLatestOffsetFilePath(params)(ugi) - result.isDefined shouldBe false + result.isFailure shouldBe true } - private def createOffsetFile(checkpointDir: Path, batchId: Int) = { - val tmpCommitsDir = Files.createDirectory(checkpointDir.resolve("offsets")) - (0 to batchId).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) + private def createOffsetFiles(maxBatchId: Int) = { + (0 to maxBatchId).map { i => + val fst = new FileStatus() + fst.setPath(new fs.Path(s"abc/def/$i")) + fst + } + }.toArray + + private def getHdfsParameters = { + new HdfsParameters( + keytab = "", + principal = "", + checkpointLocation = "/checkpoints" + ) } - private def createCommitFile(checkpointDir: Path, batchId: Int) = { - val tmpCommitsDir = Files.createDirectory(checkpointDir.resolve("commits")) - (0 to batchId).map(i => Files.createFile(tmpCommitsDir.resolve(s"$i"))) + private def createInputStream(str: String) = { + val tmpFile = Files.createTempFile(baseDirPath, "checkpoint-service-test", ".txt") + Files.write(tmpFile, str.getBytes(StandardCharsets.UTF_8)) + localFs.open(new fs.Path(tmpFile.toString), 4096) } } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala index bee6e2e15..5db7445e4 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HyperdriveOffsetComparisonServiceTest.scala @@ -15,6 +15,7 @@ package za.co.absa.hyperdrive.trigger.api.rest.services +import org.apache.hadoop.security.UserGroupInformation import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.{any, eq => eqTo} import org.mockito.Mockito.{never, reset, verify, when} @@ -24,11 +25,19 @@ import za.co.absa.hyperdrive.trigger.configuration.application.DefaultTestSparkC import za.co.absa.hyperdrive.trigger.models.enums.JobTypes import za.co.absa.hyperdrive.trigger.models.{ShellInstanceParameters, SparkInstanceParameters} +import scala.util.Try + class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers with BeforeAndAfter with MockitoSugar { private val checkpointService = mock[CheckpointService] private val kafkaService = mock[KafkaService] + private val ugiService = mock[UserGroupInformationService] + private val ugi = mock[UserGroupInformation] private val underTest = - new HyperdriveOffsetComparisonServiceImpl(DefaultTestSparkConfig().yarn, checkpointService, kafkaService) + new HyperdriveOffsetComparisonServiceImpl(DefaultTestSparkConfig().yarn, + checkpointService, + ugiService, + kafkaService + ) before { reset(checkpointService) @@ -37,13 +46,14 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers "isNewJobInstanceRequired" should "return false if the kafka and checkpoint folder offsets are the same" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, ugiService, kafkaService) val jobParameters = getJobParameters - when(checkpointService.getLatestOffsetFilePath(any())) - .thenReturn(Some(("/checkpoint/path/some-topic/offsets/21", true))) - when(checkpointService.getOffsetsFromFile(any())) - .thenReturn(Option(Map("some-topic" -> Map(2 -> 2021L, 0 -> 21L, 1 -> 1021L)))) + when(ugiService.loginUserFromKeytab(any(), any())).thenReturn(ugi) + when(checkpointService.getLatestOffsetFilePath(any())(any())) + .thenReturn(Try(Some(("/checkpoint/path/some-topic/offsets/21", true)))) + when(checkpointService.getOffsetsFromFile(any())(any())) + .thenReturn(Try(Some(Map("some-topic" -> Map(2 -> 2021L, 0 -> 21L, 1 -> 1021L))))) when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) @@ -51,11 +61,11 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers resultFut.map { result => val hdfsParametersCaptor: ArgumentCaptor[HdfsParameters] = ArgumentCaptor.forClass(classOf[HdfsParameters]) - verify(checkpointService).getLatestOffsetFilePath(hdfsParametersCaptor.capture()) + verify(checkpointService).getLatestOffsetFilePath(hdfsParametersCaptor.capture())(any()) hdfsParametersCaptor.getValue.keytab shouldBe "/path/to/keytab" hdfsParametersCaptor.getValue.principal shouldBe "principal" hdfsParametersCaptor.getValue.checkpointLocation shouldBe "/checkpoint/path/some-topic" - verify(checkpointService).getOffsetsFromFile(eqTo("/checkpoint/path/some-topic/offsets/21")) + verify(checkpointService).getOffsetsFromFile(eqTo("/checkpoint/path/some-topic/offsets/21"))(any()) verify(kafkaService).getEndOffsets(eqTo("some-topic"), any()) result shouldBe false } @@ -67,8 +77,8 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService, never()).getLatestOffsetFilePath(any()) - verify(checkpointService, never()).getOffsetsFromFile(any()) + verify(checkpointService, never()).getLatestOffsetFilePath(any())(any()) + verify(checkpointService, never()).getOffsetsFromFile(any())(any()) verify(kafkaService, never()).getEndOffsets(any(), any()) result shouldBe true } @@ -80,8 +90,8 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService, never()).getLatestOffsetFilePath(any()) - verify(checkpointService, never()).getOffsetsFromFile(any()) + verify(checkpointService, never()).getLatestOffsetFilePath(any())(any()) + verify(checkpointService, never()).getOffsetsFromFile(any())(any()) verify(kafkaService, never()).getEndOffsets(any(), any()) result shouldBe true } @@ -93,8 +103,8 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService, never()).getLatestOffsetFilePath(any()) - verify(checkpointService, never()).getOffsetsFromFile(any()) + verify(checkpointService, never()).getLatestOffsetFilePath(any())(any()) + verify(checkpointService, never()).getOffsetsFromFile(any())(any()) verify(kafkaService, never()).getEndOffsets(any(), any()) result shouldBe true } @@ -109,8 +119,8 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService, never()).getLatestOffsetFilePath(any()) - verify(checkpointService, never()).getOffsetsFromFile(any()) + verify(checkpointService, never()).getLatestOffsetFilePath(any())(any()) + verify(checkpointService, never()).getOffsetsFromFile(any())(any()) verify(kafkaService, never()).getEndOffsets(any(), any()) result shouldBe true } @@ -118,7 +128,7 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return false if the kafka topic does not exist" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, ugiService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map[Int, Long]()) @@ -127,14 +137,14 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService, never()).getLatestOffsetFilePath(any()) + verify(checkpointService, never()).getLatestOffsetFilePath(any())(any()) result shouldBe false } } it should "return false if the kafka topic is empty" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, ugiService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 42L)) @@ -143,101 +153,102 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService, never()).getLatestOffsetFilePath(any()) + verify(checkpointService, never()).getLatestOffsetFilePath(any())(any()) result shouldBe false } } it should "return true if no offset file is present" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, ugiService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) - when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(None) + when(ugiService.loginUserFromKeytab(any(), any())).thenReturn(ugi) + when(checkpointService.getLatestOffsetFilePath(any())(any())).thenReturn(Try(None)) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService).getLatestOffsetFilePath(any()) + verify(checkpointService).getLatestOffsetFilePath(any())(any()) result shouldBe true } } it should "return true if the offset is not committed" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, ugiService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) - when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", false))) + when(checkpointService.getLatestOffsetFilePath(any())(any())).thenReturn(Try(Some(("1", false)))) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService).getLatestOffsetFilePath(any()) + verify(checkpointService).getLatestOffsetFilePath(any())(any()) result shouldBe true } } it should "return true if a offset file could not be parsed" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, ugiService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) - when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(checkpointService.getOffsetsFromFile(any())) + when(checkpointService.getLatestOffsetFilePath(any())(any())).thenReturn(Try(Some(("1", true)))) + when(checkpointService.getOffsetsFromFile(any())(any())) .thenThrow(new RuntimeException("Failed to parse")) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService).getLatestOffsetFilePath(any()) - verify(checkpointService).getOffsetsFromFile(any()) + verify(checkpointService).getLatestOffsetFilePath(any())(any()) + verify(checkpointService).getOffsetsFromFile(any())(any()) result shouldBe true } } it should "return true if the checkpoints offset does not contain the topic" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, ugiService, kafkaService) val jobParameters = getJobParameters when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 100L)) - when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(checkpointService.getOffsetsFromFile(any())) - .thenReturn(Option(Map("some-other-topic" -> Map(0 -> 21L)))) + when(checkpointService.getLatestOffsetFilePath(any())(any())).thenReturn(Try(Some(("1", true)))) + when(checkpointService.getOffsetsFromFile(any())(any())) + .thenReturn(Try(Some(Map("some-other-topic" -> Map(0 -> 21L))))) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService).getLatestOffsetFilePath(any()) - verify(checkpointService).getOffsetsFromFile(any()) + verify(checkpointService).getLatestOffsetFilePath(any())(any()) + verify(checkpointService).getOffsetsFromFile(any())(any()) result shouldBe true } } it should "return true if the kafka offsets and checkpoint offset do not have the same set of partitions" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, ugiService, kafkaService) val jobParameters = getJobParameters - when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(checkpointService.getOffsetsFromFile(any())) - .thenReturn(Option(Map("some-topic" -> Map(0 -> 21L)))) + when(checkpointService.getLatestOffsetFilePath(any())(any())).thenReturn(Try(Some(("1", true)))) + when(checkpointService.getOffsetsFromFile(any())(any())) + .thenReturn(Try(Some(Map("some-topic" -> Map(0 -> 21L))))) when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 1L, 2 -> 2L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 21L, 1 -> 1021L, 2 -> 2021L)) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService).getLatestOffsetFilePath(any()) - verify(checkpointService).getOffsetsFromFile(any()) + verify(checkpointService).getLatestOffsetFilePath(any())(any()) + verify(checkpointService).getOffsetsFromFile(any())(any()) verify(kafkaService).getEndOffsets(any(), any()) result shouldBe true } @@ -245,20 +256,20 @@ class HyperdriveOffsetComparisonServiceTest extends AsyncFlatSpec with Matchers it should "return true if the kafka offsets and checkpoint offsets are not the same" in { val config = getSparkConfig - val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, kafkaService) + val underTest = new HyperdriveOffsetComparisonServiceImpl(config.yarn, checkpointService, ugiService, kafkaService) val jobParameters = getJobParameters - when(checkpointService.getLatestOffsetFilePath(any())).thenReturn(Some(("1", true))) - when(checkpointService.getOffsetsFromFile(any())) - .thenReturn(Option(Map("some-topic" -> Map(0 -> 42L, 1 -> 55L)))) + when(checkpointService.getLatestOffsetFilePath(any())(any())).thenReturn(Try(Some(("1", true)))) + when(checkpointService.getOffsetsFromFile(any())(any())) + .thenReturn(Try(Some(Map("some-topic" -> Map(0 -> 42L, 1 -> 55L))))) when(kafkaService.getBeginningOffsets(any(), any())).thenReturn(Map(0 -> 0L, 1 -> 0L)) when(kafkaService.getEndOffsets(any(), any())).thenReturn(Map(0 -> 42L, 1 -> 7L)) val resultFut = underTest.isNewJobInstanceRequired(jobParameters) resultFut.map { result => - verify(checkpointService).getLatestOffsetFilePath(any()) - verify(checkpointService).getOffsetsFromFile(any()) + verify(checkpointService).getLatestOffsetFilePath(any())(any()) + verify(checkpointService).getOffsetsFromFile(any())(any()) verify(kafkaService).getEndOffsets(any(), any()) result shouldBe true } From b04ac94d1950a36e02fc9a55f3a85b29abd3dfdd Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 21 Jul 2022 12:16:33 +0200 Subject: [PATCH 31/34] PR fix: Move parse method to HdfsService, add tests --- .../api/rest/services/CheckpointService.scala | 43 +---------- .../api/rest/services/HdfsService.scala | 45 +++++++++++- .../trigger/api/rest/utils/ScalaUtil.scala | 3 +- .../rest/services/CheckpointServiceTest.scala | 65 ++++++----------- .../api/rest/services/HdfsServiceTest.scala | 71 +++++++++++++++++++ .../api/rest/utils/ScalaUtilTest.scala | 36 ++++++++++ 6 files changed, 174 insertions(+), 89 deletions(-) create mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala create mode 100644 src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtilTest.scala diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala index b4db2d1ca..df892a782 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointService.scala @@ -17,17 +17,13 @@ package za.co.absa.hyperdrive.trigger.api.rest.services import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule -import org.apache.commons.io.IOUtils -import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} +import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.deploy.SparkHadoopUtil import org.slf4j.LoggerFactory import org.springframework.stereotype.Service import za.co.absa.hyperdrive.trigger.api.rest.utils.ScalaUtil.swap -import java.nio.charset.StandardCharsets.UTF_8 import javax.inject.Inject -import scala.io.Source import scala.util.Try trait CheckpointService { @@ -51,8 +47,6 @@ class CheckpointServiceImpl @Inject() (hdfsService: HdfsService) extends Checkpo private val mapper = new ObjectMapper().registerModule(DefaultScalaModule) private val offsetsDirName = "offsets" private val commitsDirName = "commits" - private lazy val conf = SparkHadoopUtil.get.conf - private lazy val fs = FileSystem.get(conf) /** * See org.apache.spark.sql.execution.streaming.HDFSMetadataLog @@ -72,7 +66,7 @@ class CheckpointServiceImpl @Inject() (hdfsService: HdfsService) extends Checkpo override def getOffsetsFromFile( path: String )(implicit ugi: UserGroupInformation): Try[Option[TopicPartitionOffsets]] = { - parseFileAndClose(path, parseKafkaOffsetStream) + hdfsService.parseFileAndClose(path, parseKafkaOffsetStream) } /** @@ -102,39 +96,6 @@ class CheckpointServiceImpl @Inject() (hdfsService: HdfsService) extends Checkpo } } - /** - * @param pathStr path to the file as a string - * @param parseFn function that parses the file line by line. Caution: It must materialize the content, - * because the file is closed after the method completes. E.g. it must not return an iterator. - * @tparam R type of the parsed value - * @return None if the file doesn't exist, Some with the parsed content - */ - private def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R)(implicit - ugi: UserGroupInformation - ): Try[Option[R]] = { - val path = new Path(pathStr) - hdfsService.exists(path).flatMap { exists => - if (exists) { - hdfsService.open(path).map { input => - try { - val lines = Source.fromInputStream(input, UTF_8.name()).getLines() - Some(parseFn(lines)) - } catch { - case e: Exception => - // re-throw the exception with the log file path added - throw new Exception(s"Failed to parse file $path", e) - } finally { - IOUtils.closeQuietly(input) - } - } - - } else { - logger.debug(s"Could not find file $path") - Try(None) - } - } - } - /** * see org.apache.spark.sql.execution.streaming.OffsetSeqLog * and org.apache.spark.sql.kafka010.JsonUtils diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala index 38a1d1c16..63a173827 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -15,22 +15,30 @@ package za.co.absa.hyperdrive.trigger.api.rest.services +import org.apache.commons.io.IOUtils import org.apache.hadoop.fs._ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.deploy.SparkHadoopUtil +import org.slf4j.LoggerFactory import org.springframework.stereotype.Service +import java.nio.charset.StandardCharsets.UTF_8 import java.security.PrivilegedExceptionAction +import scala.io.Source import scala.util.Try trait HdfsService { def exists(path: Path)(implicit ugi: UserGroupInformation): Try[Boolean] def open(path: Path)(implicit ugi: UserGroupInformation): Try[FSDataInputStream] def listStatus(path: Path, filter: PathFilter)(implicit ugi: UserGroupInformation): Try[Array[FileStatus]] + def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R)(implicit + ugi: UserGroupInformation + ): Try[Option[R]] } @Service class HdfsServiceImpl extends HdfsService { + private val logger = LoggerFactory.getLogger(this.getClass) private lazy val conf = SparkHadoopUtil.get.conf override def exists(path: Path)(implicit ugi: UserGroupInformation): Try[Boolean] = { Try { @@ -58,7 +66,42 @@ class HdfsServiceImpl extends HdfsService { } } - private def fs = FileSystem.get(conf) + /** + * @param pathStr path to the file as a string + * @param parseFn function that parses the file line by line. Caution: It must materialize the content, + * because the file is closed after the method completes. E.g. it must not return an iterator. + * @tparam R type of the parsed value + * @return None if the file doesn't exist, Some with the parsed content + */ + override def parseFileAndClose[R](pathStr: String, parseFn: Iterator[String] => R)(implicit + ugi: UserGroupInformation + ): Try[Option[R]] = { + for { + path <- Try(new Path(pathStr)) + exists <- exists(path) + parseResult <- + if (exists) { + open(path).map { input => + try { + val lines = Source.fromInputStream(input, UTF_8.name()).getLines() + Some(parseFn(lines)) + } catch { + case e: Exception => + // re-throw the exception with the log file path added + throw new Exception(s"Failed to parse file $path", e) + } finally { + IOUtils.closeQuietly(input) + } + } + + } else { + logger.debug(s"Could not find file $path") + Try(None) + } + } yield parseResult + } + + private lazy val fs = FileSystem.get(conf) private def doAs[T](fn: => T)(implicit ugi: UserGroupInformation) = { ugi.doAs(new PrivilegedExceptionAction[T] { diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtil.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtil.scala index 377ed77ae..de4a9ab40 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtil.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtil.scala @@ -15,11 +15,10 @@ package za.co.absa.hyperdrive.trigger.api.rest.utils -import scala.util.Try +import scala.util.{Failure, Success, Try} object ScalaUtil { def swap[T](optTry: Option[Try[T]]): Try[Option[T]] = { - import scala.util.{Failure, Success} optTry match { case Some(Success(t)) => Success(Some(t)) case Some(Failure(e)) => Failure(e) diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala index b014e9256..8f4a49ba3 100644 --- a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/CheckpointServiceTest.scala @@ -16,71 +16,60 @@ package za.co.absa.hyperdrive.trigger.api.rest.services import org.apache.hadoop.fs -import org.apache.hadoop.fs.{FileStatus, FileSystem} +import org.apache.hadoop.fs.FileStatus import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.deploy.SparkHadoopUtil +import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{reset, when} +import org.mockito.Mockito.{reset, verify, when} import org.mockito.invocation.InvocationOnMock import org.scalatest.mockito.MockitoSugar import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} -import za.co.absa.commons.io.TempDirectory -import java.nio.charset.StandardCharsets -import java.nio.file.{Files, Path} import scala.util.{Failure, Try} class CheckpointServiceTest extends FlatSpec with Matchers with BeforeAndAfter with MockitoSugar { private val hdfsService = mock[HdfsService] private val ugi = mock[UserGroupInformation] private val underTest = new CheckpointServiceImpl(hdfsService) - private var baseDir: TempDirectory = _ - private var baseDirPath: Path = _ - private lazy val conf = SparkHadoopUtil.get.conf - private val localFs = FileSystem.get(conf) before { - baseDir = TempDirectory("HdfsServiceTest").deleteOnExit() - baseDirPath = baseDir.path.toAbsolutePath reset(hdfsService) } - after { - baseDir.delete() - } - - "getOffsetFromFile" should "return None if the file does not exist" in { - when(hdfsService.exists(any())(any())).thenReturn(Try(false)) + "getOffsetFromFile" should "return None if parsing failed" in { + when(hdfsService.parseFileAndClose(any(), any())(any())).thenReturn(Try(None)) val result = underTest.getOffsetsFromFile("non-existent")(ugi) result shouldBe Try(None) } - it should "throw an exception if parsing throws an error" in { - when(hdfsService.exists(any())(any())).thenReturn(Try(true)) - when(hdfsService.open(any())(any())).thenReturn(Try(createInputStream(""))) + it should "return the parsed contents" in { + val offsets = Map( + "topic" -> Map(0 -> 1000L) + ) + when(hdfsService.parseFileAndClose[underTest.TopicPartitionOffsets](any(), any())(any())) + .thenReturn(Try(Some(offsets))) - val result = underTest.getOffsetsFromFile("some-file")(ugi) + val result = underTest.getOffsetsFromFile("any")(ugi) - result.isFailure shouldBe true - result.failed.get.getMessage should include("some-file") + result shouldBe Try(Some(offsets)) } it should "parse an offset file" in { + underTest.getOffsetsFromFile("any")(ugi) + val fnCaptor: ArgumentCaptor[Iterator[String] => underTest.TopicPartitionOffsets] = + ArgumentCaptor.forClass(classOf[Iterator[String] => underTest.TopicPartitionOffsets]) + verify(hdfsService).parseFileAndClose(any(), fnCaptor.capture())(any()) + val parseFn = fnCaptor.getValue val lines = Seq( "v1", raw"""{"batchWatermarkMs":0,"batchTimestampMs":1633360640176}""", raw"""{"my.topic":{"2":2021,"1":1021,"3":3021,"0":21}, "my.other.topic":{"0":0}}""" - ).mkString("\n") - when(hdfsService.exists(any())(any())).thenReturn(Try(true)) - when(hdfsService.open(any())(any())).thenReturn(Try(createInputStream(lines))) + ).toIterator - val resultTryOpt = underTest.getOffsetsFromFile("some-file")(ugi) + val result = parseFn.apply(lines) - resultTryOpt.isSuccess shouldBe true - resultTryOpt.get.isDefined shouldBe true - val result = resultTryOpt.get.get result.size shouldBe 2 result.head._1 shouldBe "my.topic" result.head._2 should contain theSameElementsAs Map("2" -> 2021, "1" -> 1021, "3" -> 3021, "0" -> 21) @@ -88,14 +77,6 @@ class CheckpointServiceTest extends FlatSpec with Matchers with BeforeAndAfter w result.toSeq(1)._2 should contain theSameElementsAs Map("0" -> 0) } - it should "return Failure if an exception occurred while accessing the file system" in { - when(hdfsService.exists(any())(any())).thenReturn(Failure(new RuntimeException("Failed"))) - - val result = underTest.getOffsetsFromFile("some-file")(ugi) - - result.isFailure shouldBe true - } - "getLatestOffsetFile" should "get the latest offset file, and it is committed" in { when(hdfsService.exists(any())(any())).thenReturn(Try(true)) when(hdfsService.listStatus(any(), any())(any())).thenReturn(Try(createOffsetFiles(12))) @@ -197,10 +178,4 @@ class CheckpointServiceTest extends FlatSpec with Matchers with BeforeAndAfter w checkpointLocation = "/checkpoints" ) } - - private def createInputStream(str: String) = { - val tmpFile = Files.createTempFile(baseDirPath, "checkpoint-service-test", ".txt") - Files.write(tmpFile, str.getBytes(StandardCharsets.UTF_8)) - localFs.open(new fs.Path(tmpFile.toString), 4096) - } } diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala new file mode 100644 index 000000000..e2e9a4a86 --- /dev/null +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsServiceTest.scala @@ -0,0 +1,71 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.services + +import org.apache.hadoop.security.UserGroupInformation +import org.scalatest.mockito.MockitoSugar +import org.scalatest.{BeforeAndAfter, FlatSpec, Matchers} +import za.co.absa.commons.io.TempDirectory + +import java.nio.file.{Files, Path} + +class HdfsServiceTest extends FlatSpec with BeforeAndAfter with Matchers with MockitoSugar { + + private val underTest = new HdfsServiceImpl + private val ugi = UserGroupInformation.getCurrentUser + + private var baseDir: TempDirectory = _ + private var baseDirPath: Path = _ + + before { + baseDir = TempDirectory("HdfsServiceTest").deleteOnExit() + baseDirPath = baseDir.path.toAbsolutePath + } + + after { + baseDir.delete() + } + + "parseFileAndClose" should "parse an offset file" in { + val tmpFile = Files.createTempFile(baseDirPath, "parseFileAndClose", ".txt") + val parseFn: Iterator[String] => Int = (_: Iterator[String]) => 42 + + val resultTryOpt = underTest.parseFileAndClose(tmpFile.toAbsolutePath.toString, parseFn)(ugi) + + resultTryOpt.isSuccess shouldBe true + resultTryOpt.get.isDefined shouldBe true + val result = resultTryOpt.get.get + result shouldBe 42 + } + + it should "return Failure if an exception occurred while accessing the file system" in { + val parseFn: Iterator[String] => Int = (_: Iterator[String]) => 42 + + val result = underTest.parseFileAndClose("", parseFn)(ugi) + + result.isFailure shouldBe true + } + + it should "return Failure if parsing throws an error" in { + val tmpFile = Files.createTempFile(baseDirPath, "hdfs-service-test", ".txt") + val parseFn: Iterator[String] => Int = (_: Iterator[String]) => throw new RuntimeException("Failed parsing") + val result = underTest.parseFileAndClose(tmpFile.toAbsolutePath.toString, parseFn)(ugi) + + result.isFailure shouldBe true + result.failed.get.getMessage should include("hdfs-service-test") + } +} diff --git a/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtilTest.scala b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtilTest.scala new file mode 100644 index 000000000..60b5eb320 --- /dev/null +++ b/src/test/scala/za/co/absa/hyperdrive/trigger/api/rest/utils/ScalaUtilTest.scala @@ -0,0 +1,36 @@ + +/* + * Copyright 2018 ABSA Group Limited + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package za.co.absa.hyperdrive.trigger.api.rest.utils + +import org.scalatest.{FlatSpec, Matchers} +import za.co.absa.hyperdrive.trigger.api.rest.utils.ScalaUtil.swap + +import scala.util.{Failure, Success} + +class ScalaUtilTest extends FlatSpec with Matchers { + "swap" should "swap Option[Try] to Try[Option]" in { + val ex = new RuntimeException("Failed") + val someSuccess = Some(Success(42)) + val someFailure = Some(Failure(ex)) + val none = None + + swap(someSuccess) shouldBe Success(Some(42)) + swap(someFailure) shouldBe Failure(ex) + swap(none) shouldBe Success(None) + } + +} From 61ab80b3d92a5bcfaeec44bc82e1e9214efa07aa Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 21 Jul 2022 12:22:42 +0200 Subject: [PATCH 32/34] Undo change for testing --- .../absa/hyperdrive/trigger/api/rest/services/HdfsService.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala index 63a173827..93d298be0 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -101,7 +101,7 @@ class HdfsServiceImpl extends HdfsService { } yield parseResult } - private lazy val fs = FileSystem.get(conf) + private def fs = FileSystem.get(conf) private def doAs[T](fn: => T)(implicit ugi: UserGroupInformation) = { ugi.doAs(new PrivilegedExceptionAction[T] { From 08fea77d046e8ee402c21d8acdfe59fa27d281d4 Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 21 Jul 2022 12:26:03 +0200 Subject: [PATCH 33/34] Add comment --- .../hyperdrive/trigger/api/rest/services/HdfsService.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala index 93d298be0..50789394c 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -101,6 +101,9 @@ class HdfsServiceImpl extends HdfsService { } yield parseResult } + /** + * Must not be a lazy val, because different users should get different FileSystems. FileSystem is cached internally. + */ private def fs = FileSystem.get(conf) private def doAs[T](fn: => T)(implicit ugi: UserGroupInformation) = { From 8fb2aad626980d52f0a5ec1016e7f4c6f0d0f91a Mon Sep 17 00:00:00 2001 From: Kevin Wallimann Date: Thu, 21 Jul 2022 12:29:45 +0200 Subject: [PATCH 34/34] Fix formatting --- .../absa/hyperdrive/trigger/api/rest/services/HdfsService.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala index 50789394c..b0c8d286b 100644 --- a/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala +++ b/src/main/scala/za/co/absa/hyperdrive/trigger/api/rest/services/HdfsService.scala @@ -102,7 +102,7 @@ class HdfsServiceImpl extends HdfsService { } /** - * Must not be a lazy val, because different users should get different FileSystems. FileSystem is cached internally. + * Must not be a lazy val, because different users should get different FileSystems. FileSystem is cached internally. */ private def fs = FileSystem.get(conf)