-
Notifications
You must be signed in to change notification settings - Fork 28k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-41677][CORE][SQL][SS][UI] Add Protobuf serializer for Streamin…
…gQueryProgressWrapper ### What changes were proposed in this pull request? Add Protobuf serializer for `StreamingQueryProgressWrapper ` ### Why are the changes needed? Support fast and compact serialization/deserialization for `StreamingQueryProgressWrapper ` over RocksDB. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add new UT Closes #39357 from LuciferYang/SPARK-41677. Authored-by: yangjie01 <yangjie01@baidu.com> Signed-off-by: Gengliang Wang <gengliang@apache.org>
- Loading branch information
1 parent
492356d
commit 915e9c6
Showing
10 changed files
with
537 additions
and
6 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
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
42 changes: 42 additions & 0 deletions
42
sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/SinkProgressSerializer.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,42 @@ | ||
/* | ||
* 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 org.apache.spark.sql.streaming.SinkProgress | ||
import org.apache.spark.status.protobuf.StoreTypes | ||
|
||
private[protobuf] object SinkProgressSerializer { | ||
|
||
def serialize(sink: SinkProgress): StoreTypes.SinkProgress = { | ||
val builder = StoreTypes.SinkProgress.newBuilder() | ||
builder.setDescription(sink.description) | ||
builder.setNumOutputRows(sink.numOutputRows) | ||
sink.metrics.forEach { | ||
case (k, v) => builder.putMetrics(k, v) | ||
} | ||
builder.build() | ||
} | ||
|
||
def deserialize(sink: StoreTypes.SinkProgress): SinkProgress = { | ||
new SinkProgress( | ||
description = sink.getDescription, | ||
numOutputRows = sink.getNumOutputRows, | ||
metrics = sink.getMetricsMap | ||
) | ||
} | ||
} |
65 changes: 65 additions & 0 deletions
65
sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/SourceProgressSerializer.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,65 @@ | ||
/* | ||
* 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.{List => JList} | ||
|
||
import org.apache.spark.sql.streaming.SourceProgress | ||
import org.apache.spark.status.protobuf.StoreTypes | ||
|
||
private[protobuf] object SourceProgressSerializer { | ||
|
||
def serialize(source: SourceProgress): StoreTypes.SourceProgress = { | ||
val builder = StoreTypes.SourceProgress.newBuilder() | ||
builder.setDescription(source.description) | ||
builder.setStartOffset(source.startOffset) | ||
builder.setEndOffset(source.endOffset) | ||
builder.setLatestOffset(source.latestOffset) | ||
builder.setNumInputRows(source.numInputRows) | ||
builder.setInputRowsPerSecond(source.inputRowsPerSecond) | ||
builder.setProcessedRowsPerSecond(source.processedRowsPerSecond) | ||
source.metrics.forEach { | ||
case (k, v) => builder.putMetrics(k, v) | ||
} | ||
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 = source.getDescription, | ||
startOffset = source.getStartOffset, | ||
endOffset = source.getEndOffset, | ||
latestOffset = source.getLatestOffset, | ||
numInputRows = source.getNumInputRows, | ||
inputRowsPerSecond = source.getInputRowsPerSecond, | ||
processedRowsPerSecond = source.getProcessedRowsPerSecond, | ||
metrics = source.getMetricsMap | ||
) | ||
} | ||
} |
75 changes: 75 additions & 0 deletions
75
...src/main/scala/org/apache/spark/status/protobuf/sql/StateOperatorProgressSerializer.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,75 @@ | ||
/* | ||
* 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.{List => JList} | ||
|
||
import org.apache.spark.sql.streaming.StateOperatorProgress | ||
import org.apache.spark.status.protobuf.StoreTypes | ||
|
||
object StateOperatorProgressSerializer { | ||
|
||
def serialize(stateOperator: StateOperatorProgress): StoreTypes.StateOperatorProgress = { | ||
val builder = StoreTypes.StateOperatorProgress.newBuilder() | ||
builder.setOperatorName(stateOperator.operatorName) | ||
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) | ||
stateOperator.customMetrics.forEach { | ||
case (k, v) => builder.putCustomMetrics(k, v) | ||
} | ||
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 = 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 = stateOperator.getCustomMetricsMap | ||
) | ||
} | ||
} |
89 changes: 89 additions & 0 deletions
89
...rc/main/scala/org/apache/spark/status/protobuf/sql/StreamingQueryProgressSerializer.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,89 @@ | ||
/* | ||
* 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, Map => JMap, UUID} | ||
|
||
import com.fasterxml.jackson.databind.json.JsonMapper | ||
import com.fasterxml.jackson.module.scala.DefaultScalaModule | ||
|
||
import org.apache.spark.sql.Row | ||
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema | ||
import org.apache.spark.sql.streaming.StreamingQueryProgress | ||
import org.apache.spark.status.protobuf.StoreTypes | ||
|
||
private[protobuf] object StreamingQueryProgressSerializer { | ||
|
||
private val mapper: JsonMapper = JsonMapper.builder() | ||
.addModule(DefaultScalaModule) | ||
.build() | ||
|
||
def serialize(process: StreamingQueryProgress): StoreTypes.StreamingQueryProgress = { | ||
val builder = StoreTypes.StreamingQueryProgress.newBuilder() | ||
builder.setId(process.id.toString) | ||
builder.setRunId(process.runId.toString) | ||
builder.setName(process.name) | ||
builder.setTimestamp(process.timestamp) | ||
builder.setBatchId(process.batchId) | ||
builder.setBatchDuration(process.batchDuration) | ||
process.durationMs.forEach { | ||
case (k, v) => builder.putDurationMs(k, v) | ||
} | ||
process.eventTime.forEach { | ||
case (k, v) => builder.putEventTime(k, v) | ||
} | ||
process.stateOperators.foreach( | ||
s => builder.addStateOperators(StateOperatorProgressSerializer.serialize(s))) | ||
process.sources.foreach( | ||
s => builder.addSources(SourceProgressSerializer.serialize(s)) | ||
) | ||
builder.setSink(SinkProgressSerializer.serialize(process.sink)) | ||
process.observedMetrics.forEach { | ||
case (k, v) => builder.putObservedMetrics(k, mapper.writeValueAsString(v)) | ||
} | ||
builder.build() | ||
} | ||
|
||
def deserialize(process: StoreTypes.StreamingQueryProgress): StreamingQueryProgress = { | ||
new StreamingQueryProgress( | ||
id = UUID.fromString(process.getId), | ||
runId = UUID.fromString(process.getRunId), | ||
name = process.getName, | ||
timestamp = process.getTimestamp, | ||
batchId = process.getBatchId, | ||
batchDuration = process.getBatchDuration, | ||
durationMs = process.getDurationMsMap, | ||
eventTime = process.getEventTimeMap, | ||
stateOperators = | ||
StateOperatorProgressSerializer.deserializeToArray(process.getStateOperatorsList), | ||
sources = SourceProgressSerializer.deserializeToArray(process.getSourcesList), | ||
sink = SinkProgressSerializer.deserialize(process.getSink), | ||
observedMetrics = convertToObservedMetrics(process.getObservedMetricsMap) | ||
) | ||
} | ||
|
||
private def convertToObservedMetrics(input: JMap[String, String]): JHashMap[String, Row] = { | ||
val observedMetrics = new JHashMap[String, Row](input.size()) | ||
val classType = classOf[GenericRowWithSchema] | ||
input.forEach { | ||
case (k, v) => | ||
observedMetrics.put(k, mapper.readValue(v, classType)) | ||
} | ||
observedMetrics | ||
} | ||
} |
Oops, something went wrong.