-
Notifications
You must be signed in to change notification settings - Fork 24
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
* Add streaming metrics Signed-off-by: Khor Shu Heng <khor.heng@go-jek.com> * Register source on spark conf Signed-off-by: Khor Shu Heng <khor.heng@go-jek.com> * Validate that rowsAfterValidation is not empty Signed-off-by: Khor Shu Heng <khor.heng@go-jek.com>
- Loading branch information
1 parent
f347686
commit e55a145
Showing
6 changed files
with
198 additions
and
14 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
45 changes: 45 additions & 0 deletions
45
spark/ingestion/src/main/scala/feast/ingestion/metrics/StreamingMetrics.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,45 @@ | ||
/* | ||
* SPDX-License-Identifier: Apache-2.0 | ||
* Copyright 2018-2020 The Feast Authors | ||
* | ||
* 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 | ||
* | ||
* https://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 feast.ingestion.metrics | ||
|
||
import org.apache.spark.SparkEnv | ||
import org.apache.spark.metrics.source.StreamingMetricSource | ||
import org.apache.spark.sql.streaming.StreamingQueryProgress | ||
|
||
class StreamingMetrics extends Serializable { | ||
|
||
private val metricSource: Option[StreamingMetricSource] = { | ||
val metricsSystem = SparkEnv.get.metricsSystem | ||
|
||
metricsSystem.getSourcesByName(StreamingMetricSource.sourceName) match { | ||
case Seq(head) => Some(head.asInstanceOf[StreamingMetricSource]) | ||
case _ => None | ||
} | ||
} | ||
|
||
def updateStreamingProgress( | ||
progress: StreamingQueryProgress | ||
): Unit = { | ||
metricSource.foreach(_.updateStreamingProgress(progress)) | ||
} | ||
|
||
def updateKafkaTimestamp(timestamp: Long): Unit = { | ||
metricSource.foreach(_.updateKafkaTimestamp(timestamp)) | ||
} | ||
} | ||
|
||
private object StreamingMetricsLock |
31 changes: 31 additions & 0 deletions
31
spark/ingestion/src/main/scala/org/apache/spark/metrics/AtomicGauge.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
/* | ||
* SPDX-License-Identifier: Apache-2.0 | ||
* Copyright 2018-2021 The Feast Authors | ||
* | ||
* 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 | ||
* | ||
* https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.spark.metrics | ||
|
||
import com.codahale.metrics.Gauge | ||
|
||
import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} | ||
|
||
class AtomicLongGauge(initialValue: Long = 0L) extends Gauge[Long] { | ||
val value = new AtomicLong(initialValue) | ||
override def getValue: Long = value.get() | ||
} | ||
|
||
class AtomicIntegerGauge(initialValue: Int = 0) extends Gauge[Int] { | ||
val value = new AtomicInteger(initialValue) | ||
override def getValue: Int = value.get() | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
47 changes: 47 additions & 0 deletions
47
spark/ingestion/src/main/scala/org/apache/spark/metrics/source/StreamingMetricSource.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,47 @@ | ||
/* | ||
* SPDX-License-Identifier: Apache-2.0 | ||
* Copyright 2018-2020 The Feast Authors | ||
* | ||
* 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 | ||
* | ||
* https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.spark.metrics.source | ||
|
||
import org.apache.spark.metrics.AtomicLongGauge | ||
import org.apache.spark.sql.streaming.StreamingQueryProgress | ||
|
||
class StreamingMetricSource extends BaseMetricSource { | ||
override val sourceName: String = StreamingMetricSource.sourceName | ||
|
||
private val BATCH_DURATION_GAUGE = | ||
metricRegistry.register(gaugeWithLabels("batch_duration_ms"), new AtomicLongGauge()) | ||
private val PROCESSED_ROWS_PER_SECOND_GAUGE = | ||
metricRegistry.register(gaugeWithLabels("input_rows_per_second"), new AtomicLongGauge()) | ||
private val INPUT_ROWS_PER_SECOND_GAUGE = | ||
metricRegistry.register(gaugeWithLabels("processed_rows_per_second"), new AtomicLongGauge()) | ||
private val LAST_CONSUMED_KAFKA_TIMESTAMP_GAUGE = | ||
metricRegistry.register(gaugeWithLabels("last_consumed_kafka_timestamp"), new AtomicLongGauge()) | ||
|
||
def updateStreamingProgress(progress: StreamingQueryProgress): Unit = { | ||
BATCH_DURATION_GAUGE.value.set(progress.batchDuration) | ||
INPUT_ROWS_PER_SECOND_GAUGE.value.set(progress.inputRowsPerSecond.toLong) | ||
PROCESSED_ROWS_PER_SECOND_GAUGE.value.set(progress.processedRowsPerSecond.toLong) | ||
} | ||
|
||
def updateKafkaTimestamp(timestamp: Long): Unit = { | ||
LAST_CONSUMED_KAFKA_TIMESTAMP_GAUGE.value.set(timestamp) | ||
} | ||
} | ||
|
||
object StreamingMetricSource { | ||
val sourceName = "streaming" | ||
} |