Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -765,3 +765,54 @@ message PoolData {
optional string name = 1;
repeated int64 stage_ids = 2;
}

message StateOperatorProgress {
optional string operator_name = 1;
int64 num_rows_total = 2;
int64 num_rows_updated = 3;
int64 all_updates_time_ms = 4;
int64 num_rows_removed = 5;
int64 all_removals_time_ms = 6;
int64 commit_time_ms = 7;
int64 memory_used_bytes = 8;
int64 num_rows_dropped_by_watermark = 9;
int64 num_shuffle_partitions = 10;
int64 num_state_store_instances = 11;
map<string, int64> custom_metrics = 12;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am concerned about the nullability of all these maps. Shall we check/test all of them?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

a904a27 check/test all map

699ebd1 add setJMapField function to Utils

}

message SourceProgress {
optional string description = 1;
optional string start_offset = 2;
optional string end_offset = 3;
optional string latest_offset = 4;
int64 num_input_rows = 5;
double input_rows_per_second = 6;
double processed_rows_per_second = 7;
map<string, string> metrics = 8;
}

message SinkProgress {
optional string description = 1;
int64 num_output_rows = 2;
map<string, string> metrics = 3;
}

message StreamingQueryProgress {
optional string id = 1;
optional string run_id = 2;
optional string name = 3;
optional string timestamp = 4;
int64 batch_id = 5;
int64 batch_duration = 6;
map<string, int64> duration_ms = 7;
map<string, string> event_time = 8;
repeated StateOperatorProgress state_operators = 9;
repeated SourceProgress sources = 10;
SinkProgress sink = 11;
map<string, string> observed_metrics = 12;
}

message StreamingQueryProgressWrapper {
StreamingQueryProgress progress = 1;
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.status.protobuf

import java.util.{Map => JMap}

object Utils {
def getOptional[T](condition: Boolean, result: () => T): Option[T] = if (condition) {
Some(result())
Expand All @@ -35,4 +37,10 @@ object Utils {
} else {
null
}

def setJMapField[K, V](input: JMap[K, V], putAllFunc: JMap[K, V] => Any): Unit = {
if (input != null && !input.isEmpty) {
putAllFunc(input)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,3 +18,4 @@
org.apache.spark.status.protobuf.sql.SQLExecutionUIDataSerializer
org.apache.spark.status.protobuf.sql.SparkPlanGraphWrapperSerializer
org.apache.spark.status.protobuf.sql.StreamingQueryDataSerializer
org.apache.spark.status.protobuf.sql.StreamingQueryProgressWrapperSerializer
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ import org.apache.spark.sql.streaming.SinkProgress.DEFAULT_NUM_OUTPUT_ROWS
* Information about updates made to stateful operators in a [[StreamingQuery]] during a trigger.
*/
@Evolving
class StateOperatorProgress private[sql](
class StateOperatorProgress private[spark](
val operatorName: String,
val numRowsTotal: Long,
val numRowsUpdated: Long,
Expand Down Expand Up @@ -125,7 +125,7 @@ class StateOperatorProgress private[sql](
* @since 2.1.0
*/
@Evolving
class StreamingQueryProgress private[sql](
class StreamingQueryProgress private[spark](
val id: UUID,
val runId: UUID,
val name: String,
Expand Down Expand Up @@ -190,7 +190,7 @@ class StreamingQueryProgress private[sql](
* @since 2.1.0
*/
@Evolving
class SourceProgress protected[sql](
class SourceProgress protected[spark](
val description: String,
val startOffset: String,
val endOffset: String,
Expand Down Expand Up @@ -236,7 +236,7 @@ class SourceProgress protected[sql](
* @since 2.1.0
*/
@Evolving
class SinkProgress protected[sql](
class SinkProgress protected[spark](
val description: String,
val numOutputRows: Long,
val metrics: ju.Map[String, String] = Map[String, String]().asJava) extends Serializable {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,7 @@ private[sql] case class StreamingQueryUIData(
}
}

private[sql] class StreamingQueryProgressWrapper(val progress: StreamingQueryProgress) {
private[spark] class StreamingQueryProgressWrapper(val progress: StreamingQueryProgress) {
@JsonIgnore @KVIndex
private val uniqueId: String = getUniqueId(progress.runId, progress.batchId, progress.timestamp)

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.status.protobuf.sql

import java.util.{HashMap => JHashMap}

import org.apache.spark.sql.streaming.SinkProgress
import org.apache.spark.status.protobuf.StoreTypes
import org.apache.spark.status.protobuf.Utils.{getStringField, setStringField}

private[protobuf] object SinkProgressSerializer {

def serialize(sink: SinkProgress): StoreTypes.SinkProgress = {
import org.apache.spark.status.protobuf.Utils.setJMapField
val builder = StoreTypes.SinkProgress.newBuilder()
setStringField(sink.description, builder.setDescription)
builder.setNumOutputRows(sink.numOutputRows)
setJMapField(sink.metrics, builder.putAllMetrics)
builder.build()
}

def deserialize(sink: StoreTypes.SinkProgress): SinkProgress = {
new SinkProgress(
description = getStringField(sink.hasDescription, () => sink.getDescription),
numOutputRows = sink.getNumOutputRows,
metrics = new JHashMap(sink.getMetricsMap)
)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.status.protobuf.sql

import java.util.{HashMap => JHashMap, List => JList}

import org.apache.spark.sql.streaming.SourceProgress
import org.apache.spark.status.protobuf.StoreTypes
import org.apache.spark.status.protobuf.Utils.{getStringField, setJMapField, setStringField}

private[protobuf] object SourceProgressSerializer {

def serialize(source: SourceProgress): StoreTypes.SourceProgress = {
val builder = StoreTypes.SourceProgress.newBuilder()
setStringField(source.description, builder.setDescription)
setStringField(source.startOffset, builder.setStartOffset)
setStringField(source.endOffset, builder.setEndOffset)
setStringField(source.latestOffset, builder.setLatestOffset)
builder.setNumInputRows(source.numInputRows)
builder.setInputRowsPerSecond(source.inputRowsPerSecond)
builder.setProcessedRowsPerSecond(source.processedRowsPerSecond)
setJMapField(source.metrics, builder.putAllMetrics)
builder.build()
}

def deserializeToArray(sourceList: JList[StoreTypes.SourceProgress]): Array[SourceProgress] = {
val size = sourceList.size()
val result = new Array[SourceProgress](size)
var i = 0
while (i < size) {
result(i) = deserialize(sourceList.get(i))
i += 1
}
result
}

private def deserialize(source: StoreTypes.SourceProgress): SourceProgress = {
new SourceProgress(
description = getStringField(source.hasDescription, () => source.getDescription),
startOffset = getStringField(source.hasStartOffset, () => source.getStartOffset),
endOffset = getStringField(source.hasEndOffset, () => source.getEndOffset),
latestOffset = getStringField(source.hasLatestOffset, () => source.getLatestOffset),
numInputRows = source.getNumInputRows,
inputRowsPerSecond = source.getInputRowsPerSecond,
processedRowsPerSecond = source.getProcessedRowsPerSecond,
metrics = new JHashMap(source.getMetricsMap)
)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,76 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.status.protobuf.sql

import java.util.{HashMap => JHashMap, List => JList}

import org.apache.spark.sql.streaming.StateOperatorProgress
import org.apache.spark.status.protobuf.StoreTypes
import org.apache.spark.status.protobuf.Utils.{getStringField, setStringField}

object StateOperatorProgressSerializer {

def serialize(stateOperator: StateOperatorProgress): StoreTypes.StateOperatorProgress = {
import org.apache.spark.status.protobuf.Utils.setJMapField
val builder = StoreTypes.StateOperatorProgress.newBuilder()
setStringField(stateOperator.operatorName, builder.setOperatorName)
builder.setNumRowsTotal(stateOperator.numRowsTotal)
builder.setNumRowsUpdated(stateOperator.numRowsUpdated)
builder.setAllUpdatesTimeMs(stateOperator.allUpdatesTimeMs)
builder.setNumRowsRemoved(stateOperator.numRowsRemoved)
builder.setAllRemovalsTimeMs(stateOperator.allRemovalsTimeMs)
builder.setCommitTimeMs(stateOperator.commitTimeMs)
builder.setMemoryUsedBytes(stateOperator.memoryUsedBytes)
builder.setNumRowsDroppedByWatermark(stateOperator.numRowsDroppedByWatermark)
builder.setNumShufflePartitions(stateOperator.numShufflePartitions)
builder.setNumStateStoreInstances(stateOperator.numStateStoreInstances)
setJMapField(stateOperator.customMetrics, builder.putAllCustomMetrics)
builder.build()
}

def deserializeToArray(
stateOperatorList: JList[StoreTypes.StateOperatorProgress]): Array[StateOperatorProgress] = {
val size = stateOperatorList.size()
val result = new Array[StateOperatorProgress](size)
var i = 0
while (i < size) {
result(i) = deserialize(stateOperatorList.get(i))
i += 1
}
result
}

private def deserialize(
stateOperator: StoreTypes.StateOperatorProgress): StateOperatorProgress = {
new StateOperatorProgress(
operatorName =
getStringField(stateOperator.hasOperatorName, () => stateOperator.getOperatorName),
numRowsTotal = stateOperator.getNumRowsTotal,
numRowsUpdated = stateOperator.getNumRowsUpdated,
allUpdatesTimeMs = stateOperator.getAllUpdatesTimeMs,
numRowsRemoved = stateOperator.getNumRowsRemoved,
allRemovalsTimeMs = stateOperator.getAllRemovalsTimeMs,
commitTimeMs = stateOperator.getCommitTimeMs,
memoryUsedBytes = stateOperator.getMemoryUsedBytes,
numRowsDroppedByWatermark = stateOperator.getNumRowsDroppedByWatermark,
numShufflePartitions = stateOperator.getNumShufflePartitions,
numStateStoreInstances = stateOperator.getNumStateStoreInstances,
customMetrics = new JHashMap(stateOperator.getCustomMetricsMap)
)
}
}
Loading