[GitHub] [spark] gengliangwang commented on a diff in pull request #39642: [SPARK-41677][CORE][SQL][SS][UI] Add Protobuf serializer for `StreamingQueryProgressWrapper`

2023-01-24 Thread via GitHub


gengliangwang commented on code in PR #39642:
URL: https://github.com/apache/spark/pull/39642#discussion_r1086194955


##
sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/StreamingQueryProgressSerializer.scala:
##
@@ -0,0 +1,102 @@
+/*
+ * 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
+import org.apache.spark.status.protobuf.Utils.{getStringField, setJMapField, 
setStringField}
+
+private[protobuf] object StreamingQueryProgressSerializer {
+
+  private val mapper: JsonMapper = JsonMapper.builder()
+.addModule(DefaultScalaModule)
+.build()
+
+  def serialize(process: StreamingQueryProgress): 
StoreTypes.StreamingQueryProgress = {
+val builder = StoreTypes.StreamingQueryProgress.newBuilder()
+if (process.id != null) {
+  builder.setId(process.id.toString)
+}
+if (process.runId != null) {
+  builder.setRunId(process.runId.toString)
+}
+setStringField(process.name, builder.setName)
+setStringField(process.timestamp, builder.setTimestamp)
+builder.setBatchId(process.batchId)
+builder.setBatchDuration(process.batchDuration)
+setJMapField(process.durationMs, builder.putAllDurationMs)
+setJMapField(process.eventTime, builder.putAllEventTime)
+process.stateOperators.foreach(
+  s => 
builder.addStateOperators(StateOperatorProgressSerializer.serialize(s)))
+process.sources.foreach(
+  s => builder.addSources(SourceProgressSerializer.serialize(s))
+)
+builder.setSink(SinkProgressSerializer.serialize(process.sink))
+setJMapField(process.observedMetrics, putAllObservedMetrics(builder, _))
+builder.build()
+  }
+
+  def deserialize(process: StoreTypes.StreamingQueryProgress): 
StreamingQueryProgress = {
+val id = if (process.hasId) {
+  UUID.fromString(process.getId)
+} else null
+val runId = if (process.hasId) {
+  UUID.fromString(process.getRunId)
+} else null
+new StreamingQueryProgress(
+  id = id,
+  runId = runId,
+  name = getStringField(process.hasName, () => process.getName),
+  timestamp = getStringField(process.hasTimestamp, () => 
process.getTimestamp),
+  batchId = process.getBatchId,
+  batchDuration = process.getBatchDuration,
+  durationMs = new JHashMap(process.getDurationMsMap),
+  eventTime = new JHashMap(process.getEventTimeMap),
+  stateOperators =
+
StateOperatorProgressSerializer.deserializeToArray(process.getStateOperatorsList),
+  sources = 
SourceProgressSerializer.deserializeToArray(process.getSourcesList),
+  sink = SinkProgressSerializer.deserialize(process.getSink),
+  observedMetrics = convertToObservedMetrics(process.getObservedMetricsMap)
+)
+  }
+
+  private def putAllObservedMetrics(
+  builder: StoreTypes.StreamingQueryProgress.Builder,
+  observedMetrics: JMap[String, Row]): Unit = {
+observedMetrics.forEach {
+  case (k, v) => builder.putObservedMetrics(k, 
mapper.writeValueAsString(v))

Review Comment:
   nit: Let's add one line comment to mention why we choose to encode the row 
object with json.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] gengliangwang commented on a diff in pull request #39642: [SPARK-41677][CORE][SQL][SS][UI] Add Protobuf serializer for `StreamingQueryProgressWrapper`

2023-01-24 Thread via GitHub


gengliangwang commented on code in PR #39642:
URL: https://github.com/apache/spark/pull/39642#discussion_r1086190132


##
sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/StreamingQueryProgressSerializer.scala:
##
@@ -0,0 +1,99 @@
+/*
+ * 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
+import org.apache.spark.status.protobuf.Utils.{getStringField, setStringField}
+
+private[protobuf] object StreamingQueryProgressSerializer {
+
+  private val mapper: JsonMapper = JsonMapper.builder()
+.addModule(DefaultScalaModule)
+.build()
+
+  def serialize(process: StreamingQueryProgress): 
StoreTypes.StreamingQueryProgress = {
+import org.apache.spark.status.protobuf.Utils.setJMapField
+val builder = StoreTypes.StreamingQueryProgress.newBuilder()
+if (process.id != null) {

Review Comment:
   nvm, let's keep the current code



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] gengliangwang commented on a diff in pull request #39642: [SPARK-41677][CORE][SQL][SS][UI] Add Protobuf serializer for `StreamingQueryProgressWrapper`

2023-01-24 Thread via GitHub


gengliangwang commented on code in PR #39642:
URL: https://github.com/apache/spark/pull/39642#discussion_r1086049823


##
sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/StreamingQueryProgressSerializer.scala:
##
@@ -0,0 +1,99 @@
+/*
+ * 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
+import org.apache.spark.status.protobuf.Utils.{getStringField, setStringField}
+
+private[protobuf] object StreamingQueryProgressSerializer {
+
+  private val mapper: JsonMapper = JsonMapper.builder()
+.addModule(DefaultScalaModule)
+.build()
+
+  def serialize(process: StreamingQueryProgress): 
StoreTypes.StreamingQueryProgress = {
+import org.apache.spark.status.protobuf.Utils.setJMapField
+val builder = StoreTypes.StreamingQueryProgress.newBuilder()
+if (process.id != null) {
+  builder.setId(process.id.toString)
+}
+if (process.runId != null) {
+  builder.setRunId(process.runId.toString)
+}
+setStringField(process.name, builder.setName)
+setStringField(process.timestamp, builder.setTimestamp)
+builder.setBatchId(process.batchId)
+builder.setBatchDuration(process.batchDuration)
+setJMapField(process.durationMs, builder.putAllDurationMs)
+setJMapField(process.eventTime, builder.putAllEventTime)
+process.stateOperators.foreach(
+  s => 
builder.addStateOperators(StateOperatorProgressSerializer.serialize(s)))
+process.sources.foreach(
+  s => builder.addSources(SourceProgressSerializer.serialize(s))
+)
+builder.setSink(SinkProgressSerializer.serialize(process.sink))
+if (process.observedMetrics != null && !process.observedMetrics.isEmpty) {

Review Comment:
   use setJMapField?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] gengliangwang commented on a diff in pull request #39642: [SPARK-41677][CORE][SQL][SS][UI] Add Protobuf serializer for `StreamingQueryProgressWrapper`

2023-01-24 Thread via GitHub


gengliangwang commented on code in PR #39642:
URL: https://github.com/apache/spark/pull/39642#discussion_r1086049614


##
sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/StreamingQueryProgressSerializer.scala:
##
@@ -0,0 +1,99 @@
+/*
+ * 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
+import org.apache.spark.status.protobuf.Utils.{getStringField, setStringField}
+
+private[protobuf] object StreamingQueryProgressSerializer {
+
+  private val mapper: JsonMapper = JsonMapper.builder()
+.addModule(DefaultScalaModule)
+.build()
+
+  def serialize(process: StreamingQueryProgress): 
StoreTypes.StreamingQueryProgress = {
+import org.apache.spark.status.protobuf.Utils.setJMapField
+val builder = StoreTypes.StreamingQueryProgress.newBuilder()
+if (process.id != null) {

Review Comment:
   use setStringField?



##
sql/core/src/main/scala/org/apache/spark/status/protobuf/sql/StreamingQueryProgressSerializer.scala:
##
@@ -0,0 +1,99 @@
+/*
+ * 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
+import org.apache.spark.status.protobuf.Utils.{getStringField, setStringField}
+
+private[protobuf] object StreamingQueryProgressSerializer {
+
+  private val mapper: JsonMapper = JsonMapper.builder()
+.addModule(DefaultScalaModule)
+.build()
+
+  def serialize(process: StreamingQueryProgress): 
StoreTypes.StreamingQueryProgress = {
+import org.apache.spark.status.protobuf.Utils.setJMapField
+val builder = StoreTypes.StreamingQueryProgress.newBuilder()
+if (process.id != null) {
+  builder.setId(process.id.toString)
+}
+if (process.runId != null) {
+  builder.setRunId(process.runId.toString)

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] gengliangwang commented on a diff in pull request #39642: [SPARK-41677][CORE][SQL][SS][UI] Add Protobuf serializer for `StreamingQueryProgressWrapper`

2023-01-22 Thread via GitHub


gengliangwang commented on code in PR #39642:
URL: https://github.com/apache/spark/pull/39642#discussion_r1083550779


##
core/src/main/protobuf/org/apache/spark/status/protobuf/store_types.proto:
##
@@ -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 custom_metrics = 12;

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



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org